Introduce point in time APIs in x-pack basic (#61062)
This commit introduces a new API that manages point-in-times in x-pack basic. Elasticsearch pit (point in time) is a lightweight view into the state of the data as it existed when initiated. A search request by default executes against the most recent point in time. In some cases, it is preferred to perform multiple search requests using the same point in time. For example, if refreshes happen between search_after requests, then the results of those requests might not be consistent as changes happening between searches are only visible to the more recent point in time. A point in time must be opened before being used in search requests. The `keep_alive` parameter tells Elasticsearch how long it should keep a point in time around. ``` POST /my_index/_pit?keep_alive=1m ``` The response from the above request includes a `id`, which should be passed to the `id` of the `pit` parameter of search requests. ``` POST /_search { "query": { "match" : { "title" : "elasticsearch" } }, "pit": { "id": "46ToAwMDaWR4BXV1aWQxAgZub2RlXzEAAAAAAAAAAAEBYQNpZHkFdXVpZDIrBm5vZGVfMwAAAAAAAAAAKgFjA2lkeQV1dWlkMioGbm9kZV8yAAAAAAAAAAAMAWICBXV1aWQyAAAFdXVpZDEAAQltYXRjaF9hbGw_gAAAAA==", "keep_alive": "1m" } } ``` Point-in-times are automatically closed when the `keep_alive` is elapsed. However, keeping point-in-times has a cost; hence, point-in-times should be closed as soon as they are no longer used in search requests. ``` DELETE /_pit { "id" : "46ToAwMDaWR4BXV1aWQxAgZub2RlXzEAAAAAAAAAAAEBYQNpZHkFdXVpZDIrBm5vZGVfMwAAAAAAAAAAKgFjA2lkeQV1dWlkMioGbm9kZV8yAAAAAAAAAAAMAWIBBXV1aWQyAAA=" } ``` #### Notable works in this change: - Move the search state to the coordinating node: #52741 - Allow searches with a specific reader context: #53989 - Add the ability to acquire readers in IndexShard: #54966 Relates #46523 Relates #26472 Co-authored-by: Jim Ferenczi <jimczi@apache.org>
This commit is contained in:
parent
87c889f9c9
commit
3d69b5c41e
|
@ -0,0 +1,116 @@
|
|||
[role="xpack"]
|
||||
[testenv="basic"]
|
||||
[[point-in-time]]
|
||||
==== Point in time
|
||||
|
||||
A search request by default executes against the most recent visible data of
|
||||
the target indices, which is called point in time. Elasticsearch pit (point in time)
|
||||
is a lightweight view into the state of the data as it existed when initiated.
|
||||
In some cases, it's preferred to perform multiple search requests using
|
||||
the same point in time. For example, if <<indices-refresh,refreshes>> happen between
|
||||
search_after requests, then the results of those requests might not be consistent as
|
||||
changes happening between searches are only visible to the more recent point in time.
|
||||
|
||||
A point in time must be opened explicitly before being used in search requests. The
|
||||
keep_alive parameter tells Elasticsearch how long it should keep a point in time alive,
|
||||
e.g. `?keep_alive=5m`.
|
||||
|
||||
[source,console]
|
||||
--------------------------------------------------
|
||||
POST /my-index-000001/_pit?keep_alive=1m
|
||||
--------------------------------------------------
|
||||
// TEST[setup:my_index]
|
||||
|
||||
The result from the above request includes a `id`, which should
|
||||
be passed to the `id` of the `pit` parameter of a search request.
|
||||
|
||||
[source,console]
|
||||
--------------------------------------------------
|
||||
POST /_search <1>
|
||||
{
|
||||
"size": 100,
|
||||
"query": {
|
||||
"match" : {
|
||||
"title" : "elasticsearch"
|
||||
}
|
||||
},
|
||||
"pit": {
|
||||
"id": "46ToAwMDaWR4BXV1aWQxAgZub2RlXzEAAAAAAAAAAAEBYQNpZHkFdXVpZDIrBm5vZGVfMwAAAAAAAAAAKgFjA2lkeQV1dWlkMioGbm9kZV8yAAAAAAAAAAAMAWICBXV1aWQyAAAFdXVpZDEAAQltYXRjaF9hbGw_gAAAAA==", <2>
|
||||
"keep_alive": "1m" <3>
|
||||
}
|
||||
}
|
||||
--------------------------------------------------
|
||||
// TEST[catch:missing]
|
||||
|
||||
<1> A search request with the `pit` parameter must not specify `index`, `routing`,
|
||||
and {ref}/search-request-body.html#request-body-search-preference[`preference`]
|
||||
as these parameters are copied from the point in time.
|
||||
<2> The `id` parameter tells Elasticsearch to execute the request using contexts
|
||||
from this point int time.
|
||||
<3> The `keep_alive` parameter tells Elasticsearch how long it should extend
|
||||
the time to live of the point in time.
|
||||
|
||||
IMPORTANT: The open point in time request and each subsequent search request can
|
||||
return different `id`; thus always use the most recently received `id` for the
|
||||
next search request.
|
||||
|
||||
[[point-in-time-keep-alive]]
|
||||
===== Keeping point in time alive
|
||||
The `keep_alive` parameter, which is passed to a open point in time request and
|
||||
search request, extends the time to live of the corresponding point in time.
|
||||
The value (e.g. `1m`, see <<time-units>>) does not need to be long enough to
|
||||
process all data -- it just needs to be long enough for the next request.
|
||||
|
||||
Normally, the background merge process optimizes the index by merging together
|
||||
smaller segments to create new, bigger segments. Once the smaller segments are
|
||||
no longer needed they are deleted. However, open point-in-times prevent the
|
||||
old segments from being deleted since they are still in use.
|
||||
|
||||
TIP: Keeping older segments alive means that more disk space and file handles
|
||||
are needed. Ensure that you have configured your nodes to have ample free file
|
||||
handles. See <<file-descriptors>>.
|
||||
|
||||
Additionally, if a segment contains deleted or updated documents then the
|
||||
point in time must keep track of whether each document in the segment was live at
|
||||
the time of the initial search request. Ensure that your nodes have sufficient heap
|
||||
space if you have many open point-in-times on an index that is subject to ongoing
|
||||
deletes or updates.
|
||||
|
||||
You can check how many point-in-times (i.e, search contexts) are open with the
|
||||
<<cluster-nodes-stats,nodes stats API>>:
|
||||
|
||||
[source,console]
|
||||
---------------------------------------
|
||||
GET /_nodes/stats/indices/search
|
||||
---------------------------------------
|
||||
|
||||
===== Close point in time API
|
||||
|
||||
Point-in-time is automatically closed when its `keep_alive` has
|
||||
been elapsed. However keeping point-in-times has a cost, as discussed in the
|
||||
<<point-in-time-keep-alive,previous section>>. Point-in-times should be closed
|
||||
as soon as they are no longer used in search requests.
|
||||
|
||||
[source,console]
|
||||
---------------------------------------
|
||||
DELETE /_pit
|
||||
{
|
||||
"id" : "46ToAwMDaWR4BXV1aWQxAgZub2RlXzEAAAAAAAAAAAEBYQNpZHkFdXVpZDIrBm5vZGVfMwAAAAAAAAAAKgFjA2lkeQV1dWlkMioGbm9kZV8yAAAAAAAAAAAMAWIBBXV1aWQyAAA="
|
||||
}
|
||||
---------------------------------------
|
||||
// TEST[catch:missing]
|
||||
|
||||
The API returns the following response:
|
||||
|
||||
[source,console-result]
|
||||
--------------------------------------------------
|
||||
{
|
||||
"succeeded": true, <1>
|
||||
"num_freed": 3 <2>
|
||||
}
|
||||
--------------------------------------------------
|
||||
// TESTRESPONSE[s/"succeeded": true/"succeeded": $body.succeeded/]
|
||||
// TESTRESPONSE[s/"num_freed": 3/"num_freed": $body.num_freed/]
|
||||
|
||||
<1> If true, all search contexts associated with the point-in-time id are successfully closed
|
||||
<2> The number of search contexts have been successfully closed
|
|
@ -152,12 +152,8 @@ class ParentChildInnerHitContextBuilder extends InnerHitContextBuilder {
|
|||
topDocsCollector = TopScoreDocCollector.create(topN, Integer.MAX_VALUE);
|
||||
maxScoreCollector = new MaxScoreCollector();
|
||||
}
|
||||
try {
|
||||
for (LeafReaderContext ctx : context.searcher().getIndexReader().leaves()) {
|
||||
intersect(weight, innerHitQueryWeight, MultiCollector.wrap(topDocsCollector, maxScoreCollector), ctx);
|
||||
}
|
||||
} finally {
|
||||
clearReleasables(Lifetime.COLLECTION);
|
||||
for (LeafReaderContext ctx : context.searcher().getIndexReader().leaves()) {
|
||||
intersect(weight, innerHitQueryWeight, MultiCollector.wrap(topDocsCollector, maxScoreCollector), ctx);
|
||||
}
|
||||
TopDocs topDocs = topDocsCollector.topDocs(from(), size());
|
||||
float maxScore = Float.NaN;
|
||||
|
|
|
@ -37,7 +37,7 @@
|
|||
search.max_keep_alive: "1m"
|
||||
|
||||
- do:
|
||||
catch: /.*Keep alive for scroll.*is too large.*/
|
||||
catch: /.*Keep alive for.*is too large.*/
|
||||
search:
|
||||
rest_total_hits_as_int: true
|
||||
index: test_scroll
|
||||
|
@ -64,7 +64,7 @@
|
|||
- length: {hits.hits: 1 }
|
||||
|
||||
- do:
|
||||
catch: /.*Keep alive for scroll.*is too large.*/
|
||||
catch: /.*Keep alive for.*is too large.*/
|
||||
scroll:
|
||||
rest_total_hits_as_int: true
|
||||
scroll_id: $scroll_id
|
||||
|
|
|
@ -604,7 +604,7 @@ public class SearchScrollIT extends ESIntegTestCase {
|
|||
IllegalArgumentException illegalArgumentException =
|
||||
(IllegalArgumentException) ExceptionsHelper.unwrap(exc, IllegalArgumentException.class);
|
||||
assertNotNull(illegalArgumentException);
|
||||
assertThat(illegalArgumentException.getMessage(), containsString("Keep alive for scroll (2h) is too large"));
|
||||
assertThat(illegalArgumentException.getMessage(), containsString("Keep alive for request (2h) is too large"));
|
||||
|
||||
SearchResponse searchResponse = client().prepareSearch()
|
||||
.setQuery(matchAllQuery())
|
||||
|
@ -621,7 +621,7 @@ public class SearchScrollIT extends ESIntegTestCase {
|
|||
illegalArgumentException =
|
||||
(IllegalArgumentException) ExceptionsHelper.unwrap(exc, IllegalArgumentException.class);
|
||||
assertNotNull(illegalArgumentException);
|
||||
assertThat(illegalArgumentException.getMessage(), containsString("Keep alive for scroll (3h) is too large"));
|
||||
assertThat(illegalArgumentException.getMessage(), containsString("Keep alive for request (3h) is too large"));
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.logging.log4j.message.ParameterizedMessage;
|
|||
import org.apache.lucene.util.SetOnce;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.NoShardAvailableActionException;
|
||||
import org.elasticsearch.action.ShardOperationFailedException;
|
||||
|
@ -163,7 +164,7 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
|
|||
// total hits is null in the response if the tracking of total hits is disabled
|
||||
boolean withTotalHits = trackTotalHitsUpTo != SearchContext.TRACK_TOTAL_HITS_DISABLED;
|
||||
listener.onResponse(new SearchResponse(InternalSearchResponse.empty(withTotalHits), null, 0, 0, 0, buildTookInMillis(),
|
||||
ShardSearchFailure.EMPTY_ARRAY, clusters));
|
||||
ShardSearchFailure.EMPTY_ARRAY, clusters, null));
|
||||
return;
|
||||
}
|
||||
executePhase(this);
|
||||
|
@ -514,22 +515,29 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
|
|||
return request;
|
||||
}
|
||||
|
||||
protected final SearchResponse buildSearchResponse(InternalSearchResponse internalSearchResponse,
|
||||
String scrollId,
|
||||
ShardSearchFailure[] failures) {
|
||||
protected final SearchResponse buildSearchResponse(InternalSearchResponse internalSearchResponse, ShardSearchFailure[] failures,
|
||||
String scrollId, String searchContextId) {
|
||||
return new SearchResponse(internalSearchResponse, scrollId, getNumShards(), successfulOps.get(),
|
||||
skippedOps.get(), buildTookInMillis(), failures, clusters);
|
||||
skippedOps.get(), buildTookInMillis(), failures, clusters, searchContextId);
|
||||
}
|
||||
|
||||
boolean includeSearchContextInResponse() {
|
||||
return request.pointInTimeBuilder() != null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void sendSearchResponse(InternalSearchResponse internalSearchResponse, String scrollId) {
|
||||
public void sendSearchResponse(InternalSearchResponse internalSearchResponse, AtomicArray<SearchPhaseResult> queryResults) {
|
||||
ShardSearchFailure[] failures = buildShardFailures();
|
||||
Boolean allowPartialResults = request.allowPartialSearchResults();
|
||||
assert allowPartialResults != null : "SearchRequest missing setting for allowPartialSearchResults";
|
||||
if (allowPartialResults == false && failures.length > 0){
|
||||
if (request.pointInTimeBuilder() == null && allowPartialResults == false && failures.length > 0) {
|
||||
raisePhaseFailure(new SearchPhaseExecutionException("", "Shard failures", null, failures));
|
||||
} else {
|
||||
listener.onResponse(buildSearchResponse(internalSearchResponse, scrollId, failures));
|
||||
final Version minNodeVersion = clusterState.nodes().getMinNodeVersion();
|
||||
final String scrollId = request.scroll() != null ? TransportSearchHelper.buildScrollId(queryResults, minNodeVersion) : null;
|
||||
final String searchContextId =
|
||||
includeSearchContextInResponse() ? SearchContextId.encode(queryResults.asList(), aliasFilter, minNodeVersion) : null;
|
||||
listener.onResponse(buildSearchResponse(internalSearchResponse, failures, scrollId, searchContextId));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -598,12 +606,13 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
|
|||
final String[] routings = indexRoutings.getOrDefault(indexName, Collections.emptySet())
|
||||
.toArray(new String[0]);
|
||||
ShardSearchRequest shardRequest = new ShardSearchRequest(shardIt.getOriginalIndices(), request, shardIt.shardId(), getNumShards(),
|
||||
filter, indexBoost, timeProvider.getAbsoluteStartMillis(), shardIt.getClusterAlias(), routings);
|
||||
filter, indexBoost, timeProvider.getAbsoluteStartMillis(), shardIt.getClusterAlias(), routings,
|
||||
shardIt.getSearchContextId(), shardIt.getSearchContextKeepAlive());
|
||||
// if we already received a search result we can inform the shard that it
|
||||
// can return a null response if the request rewrites to match none rather
|
||||
// than creating an empty response in the search thread pool.
|
||||
// Note that, we have to disable this shortcut for scroll queries.
|
||||
shardRequest.canReturnNullResponseIfMatchNoDocs(hasShardResponse.get() && request.scroll() == null);
|
||||
// Note that, we have to disable this shortcut for queries that create a context (scroll and search context).
|
||||
shardRequest.canReturnNullResponseIfMatchNoDocs(hasShardResponse.get() && shardRequest.scroll() == null);
|
||||
return shardRequest;
|
||||
}
|
||||
|
||||
|
@ -673,8 +682,4 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
|
|||
return toExecute;
|
||||
}
|
||||
}
|
||||
|
||||
protected ClusterState clusterState() {
|
||||
return clusterState;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,20 +21,28 @@ package org.elasticsearch.action.search;
|
|||
import org.apache.logging.log4j.Logger;
|
||||
import org.apache.logging.log4j.message.ParameterizedMessage;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.StepListener;
|
||||
import org.elasticsearch.action.support.GroupedActionListener;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.util.concurrent.CountDown;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.elasticsearch.transport.TransportResponse;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.function.BiFunction;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.elasticsearch.action.search.TransportSearchHelper.parseScrollId;
|
||||
|
||||
final class ClearScrollController implements Runnable {
|
||||
public final class ClearScrollController implements Runnable {
|
||||
private final DiscoveryNodes nodes;
|
||||
private final SearchTransportService searchTransportService;
|
||||
private final CountDown expectedOps;
|
||||
|
@ -56,19 +64,18 @@ final class ClearScrollController implements Runnable {
|
|||
expectedOps = nodes.getSize();
|
||||
runner = this::cleanAllScrolls;
|
||||
} else {
|
||||
List<ScrollIdForNode> parsedScrollIds = new ArrayList<>();
|
||||
for (String parsedScrollId : request.getScrollIds()) {
|
||||
ScrollIdForNode[] context = parseScrollId(parsedScrollId).getContext();
|
||||
for (ScrollIdForNode id : context) {
|
||||
parsedScrollIds.add(id);
|
||||
}
|
||||
// TODO: replace this with #closeContexts
|
||||
List<SearchContextIdForNode> contexts = new ArrayList<>();
|
||||
for (String scrollId : request.getScrollIds()) {
|
||||
SearchContextIdForNode[] context = parseScrollId(scrollId).getContext();
|
||||
Collections.addAll(contexts, context);
|
||||
}
|
||||
if (parsedScrollIds.isEmpty()) {
|
||||
if (contexts.isEmpty()) {
|
||||
expectedOps = 0;
|
||||
runner = () -> listener.onResponse(new ClearScrollResponse(true, 0));
|
||||
} else {
|
||||
expectedOps = parsedScrollIds.size();
|
||||
runner = () -> cleanScrollIds(parsedScrollIds);
|
||||
expectedOps = contexts.size();
|
||||
runner = () -> cleanScrollIds(contexts);
|
||||
}
|
||||
}
|
||||
this.expectedOps = new CountDown(expectedOps);
|
||||
|
@ -101,17 +108,17 @@ final class ClearScrollController implements Runnable {
|
|||
}
|
||||
}
|
||||
|
||||
void cleanScrollIds(List<ScrollIdForNode> parsedScrollIds) {
|
||||
SearchScrollAsyncAction.collectNodesAndRun(parsedScrollIds, nodes, searchTransportService, ActionListener.wrap(
|
||||
void cleanScrollIds(List<SearchContextIdForNode> contextIds) {
|
||||
SearchScrollAsyncAction.collectNodesAndRun(contextIds, nodes, searchTransportService, ActionListener.wrap(
|
||||
lookup -> {
|
||||
for (ScrollIdForNode target : parsedScrollIds) {
|
||||
for (SearchContextIdForNode target : contextIds) {
|
||||
final DiscoveryNode node = lookup.apply(target.getClusterAlias(), target.getNode());
|
||||
if (node == null) {
|
||||
onFreedContext(false);
|
||||
} else {
|
||||
try {
|
||||
Transport.Connection connection = searchTransportService.getConnection(target.getClusterAlias(), node);
|
||||
searchTransportService.sendFreeContext(connection, target.getContextId(),
|
||||
searchTransportService.sendFreeContext(connection, target.getSearchContextId(),
|
||||
ActionListener.wrap(freed -> onFreedContext(freed.isFreed()), e -> onFailedFreedContext(e, node)));
|
||||
} catch (Exception e) {
|
||||
onFailedFreedContext(e, node);
|
||||
|
@ -142,4 +149,45 @@ final class ClearScrollController implements Runnable {
|
|||
listener.onResponse(new ClearScrollResponse(false, freedSearchContexts.get()));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Closes the given context id and reports the number of freed contexts via the listener
|
||||
*/
|
||||
public static void closeContexts(DiscoveryNodes nodes, SearchTransportService searchTransportService,
|
||||
Collection<SearchContextIdForNode> contextIds,
|
||||
ActionListener<Integer> listener) {
|
||||
if (contextIds.isEmpty()) {
|
||||
listener.onResponse(0);
|
||||
return;
|
||||
}
|
||||
final Set<String> clusters = contextIds.stream()
|
||||
.filter(ctx -> Strings.isEmpty(ctx.getClusterAlias()) == false)
|
||||
.map(SearchContextIdForNode::getClusterAlias).collect(Collectors.toSet());
|
||||
final StepListener<BiFunction<String, String, DiscoveryNode>> lookupListener = new StepListener<>();
|
||||
if (clusters.isEmpty() == false) {
|
||||
searchTransportService.getRemoteClusterService().collectNodes(clusters, lookupListener);
|
||||
} else {
|
||||
lookupListener.onResponse((cluster, nodeId) -> nodes.get(nodeId));
|
||||
}
|
||||
lookupListener.whenComplete(nodeLookup -> {
|
||||
final GroupedActionListener<Boolean> groupedListener = new GroupedActionListener<>(
|
||||
ActionListener.delegateFailure(listener, (l, rs) -> l.onResponse(Math.toIntExact(rs.stream().filter(r -> r).count()))),
|
||||
contextIds.size()
|
||||
);
|
||||
for (SearchContextIdForNode contextId : contextIds) {
|
||||
final DiscoveryNode node = nodeLookup.apply(contextId.getClusterAlias(), contextId.getNode());
|
||||
if (node == null) {
|
||||
groupedListener.onResponse(false);
|
||||
} else {
|
||||
try {
|
||||
final Transport.Connection connection = searchTransportService.getConnection(contextId.getClusterAlias(), node);
|
||||
searchTransportService.sendFreeContext(connection, contextId.getSearchContextId(),
|
||||
ActionListener.wrap(r -> groupedListener.onResponse(r.isFreed()), e -> groupedListener.onResponse(false)));
|
||||
} catch (Exception e) {
|
||||
groupedListener.onResponse(false);
|
||||
}
|
||||
}
|
||||
}
|
||||
}, listener::onFailure);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
package org.elasticsearch.action.search;
|
||||
|
||||
import org.apache.logging.log4j.message.ParameterizedMessage;
|
||||
import org.elasticsearch.common.util.concurrent.AtomicArray;
|
||||
import org.elasticsearch.search.SearchPhaseResult;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.dfs.AggregatedDfs;
|
||||
|
@ -41,15 +40,16 @@ import java.util.function.Function;
|
|||
* @see CountedCollector#onFailure(int, SearchShardTarget, Exception)
|
||||
*/
|
||||
final class DfsQueryPhase extends SearchPhase {
|
||||
private final QueryPhaseResultConsumer queryResult;
|
||||
private final SearchPhaseController searchPhaseController;
|
||||
private final AtomicArray<DfsSearchResult> dfsSearchResults;
|
||||
private final ArraySearchPhaseResults<SearchPhaseResult> queryResult;
|
||||
private final List<DfsSearchResult> searchResults;
|
||||
private final AggregatedDfs dfs;
|
||||
private final Function<ArraySearchPhaseResults<SearchPhaseResult>, SearchPhase> nextPhaseFactory;
|
||||
private final SearchPhaseContext context;
|
||||
private final SearchTransportService searchTransportService;
|
||||
private final SearchProgressListener progressListener;
|
||||
|
||||
DfsQueryPhase(AtomicArray<DfsSearchResult> dfsSearchResults,
|
||||
DfsQueryPhase(List<DfsSearchResult> searchResults,
|
||||
AggregatedDfs dfs,
|
||||
SearchPhaseController searchPhaseController,
|
||||
Function<ArraySearchPhaseResults<SearchPhaseResult>, SearchPhase> nextPhaseFactory,
|
||||
SearchPhaseContext context, Consumer<Exception> onPartialMergeFailure) {
|
||||
|
@ -57,8 +57,8 @@ final class DfsQueryPhase extends SearchPhase {
|
|||
this.progressListener = context.getTask().getProgressListener();
|
||||
this.queryResult = searchPhaseController.newSearchPhaseResults(context, progressListener,
|
||||
context.getRequest(), context.getNumShards(), onPartialMergeFailure);
|
||||
this.searchPhaseController = searchPhaseController;
|
||||
this.dfsSearchResults = dfsSearchResults;
|
||||
this.searchResults = searchResults;
|
||||
this.dfs = dfs;
|
||||
this.nextPhaseFactory = nextPhaseFactory;
|
||||
this.context = context;
|
||||
this.searchTransportService = context.getSearchTransport();
|
||||
|
@ -68,16 +68,15 @@ final class DfsQueryPhase extends SearchPhase {
|
|||
public void run() throws IOException {
|
||||
// TODO we can potentially also consume the actual per shard results from the initial phase here in the aggregateDfs
|
||||
// to free up memory early
|
||||
final List<DfsSearchResult> resultList = dfsSearchResults.asList();
|
||||
final AggregatedDfs dfs = searchPhaseController.aggregateDfs(resultList);
|
||||
final CountedCollector<SearchPhaseResult> counter = new CountedCollector<>(queryResult,
|
||||
resultList.size(),
|
||||
final CountedCollector<SearchPhaseResult> counter = new CountedCollector<>(
|
||||
queryResult,
|
||||
searchResults.size(),
|
||||
() -> context.executeNextPhase(this, nextPhaseFactory.apply(queryResult)), context);
|
||||
for (final DfsSearchResult dfsResult : resultList) {
|
||||
for (final DfsSearchResult dfsResult : searchResults) {
|
||||
final SearchShardTarget searchShardTarget = dfsResult.getSearchShardTarget();
|
||||
Transport.Connection connection = context.getConnection(searchShardTarget.getClusterAlias(), searchShardTarget.getNodeId());
|
||||
QuerySearchRequest querySearchRequest = new QuerySearchRequest(searchShardTarget.getOriginalIndices(),
|
||||
dfsResult.getContextId(), dfs);
|
||||
dfsResult.getContextId(), dfsResult.getShardSearchRequest(), dfs);
|
||||
final int shardIndex = dfsResult.getShardIndex();
|
||||
searchTransportService.sendExecuteQuery(connection, querySearchRequest, context.getTask(),
|
||||
new SearchActionListener<QuerySearchResult>(searchShardTarget, shardIndex) {
|
||||
|
|
|
@ -20,12 +20,14 @@
|
|||
package org.elasticsearch.action.search;
|
||||
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.common.util.concurrent.AtomicArray;
|
||||
import org.elasticsearch.index.query.BoolQueryBuilder;
|
||||
import org.elasticsearch.index.query.InnerHitBuilder;
|
||||
import org.elasticsearch.index.query.QueryBuilder;
|
||||
import org.elasticsearch.index.query.QueryBuilders;
|
||||
import org.elasticsearch.search.SearchHit;
|
||||
import org.elasticsearch.search.SearchHits;
|
||||
import org.elasticsearch.search.SearchPhaseResult;
|
||||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||
import org.elasticsearch.search.collapse.CollapseBuilder;
|
||||
import org.elasticsearch.search.internal.InternalSearchResponse;
|
||||
|
@ -42,13 +44,13 @@ import java.util.List;
|
|||
final class ExpandSearchPhase extends SearchPhase {
|
||||
private final SearchPhaseContext context;
|
||||
private final InternalSearchResponse searchResponse;
|
||||
private final String scrollId;
|
||||
private final AtomicArray<SearchPhaseResult> queryResults;
|
||||
|
||||
ExpandSearchPhase(SearchPhaseContext context, InternalSearchResponse searchResponse, String scrollId) {
|
||||
ExpandSearchPhase(SearchPhaseContext context, InternalSearchResponse searchResponse, AtomicArray<SearchPhaseResult> queryResults) {
|
||||
super("expand");
|
||||
this.context = context;
|
||||
this.searchResponse = searchResponse;
|
||||
this.scrollId = scrollId;
|
||||
this.queryResults = queryResults;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -110,11 +112,11 @@ final class ExpandSearchPhase extends SearchPhase {
|
|||
hit.getInnerHits().put(innerHitBuilder.getName(), innerHits);
|
||||
}
|
||||
}
|
||||
context.sendSearchResponse(searchResponse, scrollId);
|
||||
context.sendSearchResponse(searchResponse, queryResults);
|
||||
}, context::onFailure)
|
||||
);
|
||||
} else {
|
||||
context.sendSearchResponse(searchResponse, scrollId);
|
||||
context.sendSearchResponse(searchResponse, queryResults);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -22,17 +22,18 @@ import com.carrotsearch.hppc.IntArrayList;
|
|||
import org.apache.logging.log4j.Logger;
|
||||
import org.apache.logging.log4j.message.ParameterizedMessage;
|
||||
import org.apache.lucene.search.ScoreDoc;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.OriginalIndices;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
|
||||
import org.elasticsearch.common.util.concurrent.AtomicArray;
|
||||
import org.elasticsearch.search.RescoreDocIds;
|
||||
import org.elasticsearch.search.SearchPhaseResult;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.dfs.AggregatedDfs;
|
||||
import org.elasticsearch.search.fetch.FetchSearchResult;
|
||||
import org.elasticsearch.search.fetch.ShardFetchSearchRequest;
|
||||
import org.elasticsearch.search.internal.InternalSearchResponse;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchRequest;
|
||||
import org.elasticsearch.search.query.QuerySearchResult;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
|
||||
|
@ -47,26 +48,26 @@ final class FetchSearchPhase extends SearchPhase {
|
|||
private final ArraySearchPhaseResults<FetchSearchResult> fetchResults;
|
||||
private final SearchPhaseController searchPhaseController;
|
||||
private final AtomicArray<SearchPhaseResult> queryResults;
|
||||
private final BiFunction<InternalSearchResponse, String, SearchPhase> nextPhaseFactory;
|
||||
private final BiFunction<InternalSearchResponse, AtomicArray<SearchPhaseResult>, SearchPhase> nextPhaseFactory;
|
||||
private final SearchPhaseContext context;
|
||||
private final Logger logger;
|
||||
private final SearchPhaseResults<SearchPhaseResult> resultConsumer;
|
||||
private final SearchProgressListener progressListener;
|
||||
private final ClusterState clusterState;
|
||||
private final AggregatedDfs aggregatedDfs;
|
||||
|
||||
FetchSearchPhase(SearchPhaseResults<SearchPhaseResult> resultConsumer,
|
||||
SearchPhaseController searchPhaseController,
|
||||
SearchPhaseContext context,
|
||||
ClusterState clusterState) {
|
||||
this(resultConsumer, searchPhaseController, context, clusterState,
|
||||
(response, scrollId) -> new ExpandSearchPhase(context, response, scrollId));
|
||||
AggregatedDfs aggregatedDfs,
|
||||
SearchPhaseContext context) {
|
||||
this(resultConsumer, searchPhaseController, aggregatedDfs, context,
|
||||
(response, queryPhaseResults) -> new ExpandSearchPhase(context, response, queryPhaseResults));
|
||||
}
|
||||
|
||||
FetchSearchPhase(SearchPhaseResults<SearchPhaseResult> resultConsumer,
|
||||
SearchPhaseController searchPhaseController,
|
||||
AggregatedDfs aggregatedDfs,
|
||||
SearchPhaseContext context,
|
||||
ClusterState clusterState,
|
||||
BiFunction<InternalSearchResponse, String, SearchPhase> nextPhaseFactory) {
|
||||
BiFunction<InternalSearchResponse, AtomicArray<SearchPhaseResult>, SearchPhase> nextPhaseFactory) {
|
||||
super("fetch");
|
||||
if (context.getNumShards() != resultConsumer.getNumShards()) {
|
||||
throw new IllegalStateException("number of shards must match the length of the query results but doesn't:"
|
||||
|
@ -75,12 +76,12 @@ final class FetchSearchPhase extends SearchPhase {
|
|||
this.fetchResults = new ArraySearchPhaseResults<>(resultConsumer.getNumShards());
|
||||
this.searchPhaseController = searchPhaseController;
|
||||
this.queryResults = resultConsumer.getAtomicArray();
|
||||
this.aggregatedDfs = aggregatedDfs;
|
||||
this.nextPhaseFactory = nextPhaseFactory;
|
||||
this.context = context;
|
||||
this.logger = context.getLogger();
|
||||
this.resultConsumer = resultConsumer;
|
||||
this.progressListener = context.getTask().getProgressListener();
|
||||
this.clusterState = clusterState;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -105,17 +106,10 @@ final class FetchSearchPhase extends SearchPhase {
|
|||
final int numShards = context.getNumShards();
|
||||
final boolean isScrollSearch = context.getRequest().scroll() != null;
|
||||
final List<SearchPhaseResult> phaseResults = queryResults.asList();
|
||||
final String scrollId;
|
||||
if (isScrollSearch) {
|
||||
final boolean includeContextUUID = clusterState.nodes().getMinNodeVersion().onOrAfter(Version.V_7_7_0);
|
||||
scrollId = TransportSearchHelper.buildScrollId(queryResults, includeContextUUID);
|
||||
} else {
|
||||
scrollId = null;
|
||||
}
|
||||
final SearchPhaseController.ReducedQueryPhase reducedQueryPhase = resultConsumer.reduce();
|
||||
final boolean queryAndFetchOptimization = queryResults.length() == 1;
|
||||
final Runnable finishPhase = ()
|
||||
-> moveToNextPhase(searchPhaseController, scrollId, reducedQueryPhase, queryAndFetchOptimization ?
|
||||
-> moveToNextPhase(searchPhaseController, queryResults, reducedQueryPhase, queryAndFetchOptimization ?
|
||||
queryResults : fetchResults.getAtomicArray());
|
||||
if (queryAndFetchOptimization) {
|
||||
assert phaseResults.isEmpty() || phaseResults.get(0).fetchResult() != null : "phaseResults empty [" + phaseResults.isEmpty()
|
||||
|
@ -157,7 +151,8 @@ final class FetchSearchPhase extends SearchPhase {
|
|||
Transport.Connection connection = context.getConnection(searchShardTarget.getClusterAlias(),
|
||||
searchShardTarget.getNodeId());
|
||||
ShardFetchSearchRequest fetchSearchRequest = createFetchRequest(queryResult.queryResult().getContextId(), i, entry,
|
||||
lastEmittedDocPerShard, searchShardTarget.getOriginalIndices());
|
||||
lastEmittedDocPerShard, searchShardTarget.getOriginalIndices(), queryResult.getShardSearchRequest(),
|
||||
queryResult.getRescoreDocIds());
|
||||
executeFetch(i, searchShardTarget, counter, fetchSearchRequest, queryResult.queryResult(),
|
||||
connection);
|
||||
}
|
||||
|
@ -166,10 +161,12 @@ final class FetchSearchPhase extends SearchPhase {
|
|||
}
|
||||
}
|
||||
|
||||
protected ShardFetchSearchRequest createFetchRequest(SearchContextId contextId, int index, IntArrayList entry,
|
||||
ScoreDoc[] lastEmittedDocPerShard, OriginalIndices originalIndices) {
|
||||
protected ShardFetchSearchRequest createFetchRequest(ShardSearchContextId contextId, int index, IntArrayList entry,
|
||||
ScoreDoc[] lastEmittedDocPerShard, OriginalIndices originalIndices,
|
||||
ShardSearchRequest shardSearchRequest, RescoreDocIds rescoreDocIds) {
|
||||
final ScoreDoc lastEmittedDoc = (lastEmittedDocPerShard != null) ? lastEmittedDocPerShard[index] : null;
|
||||
return new ShardFetchSearchRequest(originalIndices, contextId, entry, lastEmittedDoc);
|
||||
return new ShardFetchSearchRequest(originalIndices, contextId, shardSearchRequest, entry, lastEmittedDoc,
|
||||
rescoreDocIds, aggregatedDfs);
|
||||
}
|
||||
|
||||
private void executeFetch(final int shardIndex, final SearchShardTarget shardTarget,
|
||||
|
@ -211,7 +208,9 @@ final class FetchSearchPhase extends SearchPhase {
|
|||
private void releaseIrrelevantSearchContext(QuerySearchResult queryResult) {
|
||||
// we only release search context that we did not fetch from if we are not scrolling
|
||||
// and if it has at lease one hit that didn't make it to the global topDocs
|
||||
if (context.getRequest().scroll() == null && queryResult.hasSearchContext()) {
|
||||
if (context.getRequest().scroll() == null &&
|
||||
context.getRequest().pointInTimeBuilder() == null &&
|
||||
queryResult.hasSearchContext()) {
|
||||
try {
|
||||
SearchShardTarget searchShardTarget = queryResult.getSearchShardTarget();
|
||||
Transport.Connection connection = context.getConnection(searchShardTarget.getClusterAlias(), searchShardTarget.getNodeId());
|
||||
|
@ -223,10 +222,11 @@ final class FetchSearchPhase extends SearchPhase {
|
|||
}
|
||||
|
||||
private void moveToNextPhase(SearchPhaseController searchPhaseController,
|
||||
String scrollId, SearchPhaseController.ReducedQueryPhase reducedQueryPhase,
|
||||
AtomicArray<SearchPhaseResult> queryPhaseResults,
|
||||
SearchPhaseController.ReducedQueryPhase reducedQueryPhase,
|
||||
AtomicArray<? extends SearchPhaseResult> fetchResultsArr) {
|
||||
final InternalSearchResponse internalResponse = searchPhaseController.merge(context.getRequest().scroll() != null,
|
||||
reducedQueryPhase, fetchResultsArr.asList(), fetchResultsArr::get);
|
||||
context.executeNextPhase(this, nextPhaseFactory.apply(internalResponse, scrollId));
|
||||
context.executeNextPhase(this, nextPhaseFactory.apply(internalResponse, queryPhaseResults));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -29,9 +29,9 @@ class ParsedScrollId {
|
|||
|
||||
private final String type;
|
||||
|
||||
private final ScrollIdForNode[] context;
|
||||
private final SearchContextIdForNode[] context;
|
||||
|
||||
ParsedScrollId(String source, String type, ScrollIdForNode[] context) {
|
||||
ParsedScrollId(String source, String type, SearchContextIdForNode[] context) {
|
||||
this.source = source;
|
||||
this.type = type;
|
||||
this.context = context;
|
||||
|
@ -45,7 +45,7 @@ class ParsedScrollId {
|
|||
return type;
|
||||
}
|
||||
|
||||
public ScrollIdForNode[] getContext() {
|
||||
public SearchContextIdForNode[] getContext() {
|
||||
return context;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,116 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.action.search;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.io.stream.ByteBufferStreamInput;
|
||||
import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
||||
import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput;
|
||||
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.search.SearchPhaseResult;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.internal.AliasFilter;
|
||||
import org.elasticsearch.transport.RemoteClusterAware;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Base64;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
public class SearchContextId {
|
||||
private final Map<ShardId, SearchContextIdForNode> shards;
|
||||
private final Map<String, AliasFilter> aliasFilter;
|
||||
|
||||
private SearchContextId(Map<ShardId, SearchContextIdForNode> shards, Map<String, AliasFilter> aliasFilter) {
|
||||
this.shards = shards;
|
||||
this.aliasFilter = aliasFilter;
|
||||
}
|
||||
|
||||
public Map<ShardId, SearchContextIdForNode> shards() {
|
||||
return shards;
|
||||
}
|
||||
|
||||
public Map<String, AliasFilter> aliasFilter() {
|
||||
return aliasFilter;
|
||||
}
|
||||
|
||||
public static String encode(List<SearchPhaseResult> searchPhaseResults, Map<String, AliasFilter> aliasFilter, Version version) {
|
||||
final Map<ShardId, SearchContextIdForNode> shards = new HashMap<>();
|
||||
for (SearchPhaseResult searchPhaseResult : searchPhaseResults) {
|
||||
final SearchShardTarget target = searchPhaseResult.getSearchShardTarget();
|
||||
shards.put(target.getShardId(),
|
||||
new SearchContextIdForNode(target.getClusterAlias(), target.getNodeId(), searchPhaseResult.getContextId()));
|
||||
}
|
||||
try (BytesStreamOutput out = new BytesStreamOutput()) {
|
||||
out.setVersion(version);
|
||||
Version.writeVersion(version, out);
|
||||
out.writeMap(shards, (o, k) -> k.writeTo(o), (o, v) -> v.writeTo(o));
|
||||
out.writeMap(aliasFilter, StreamOutput::writeString, (o, v) -> v.writeTo(o));
|
||||
return Base64.getUrlEncoder().encodeToString(BytesReference.toBytes(out.bytes()));
|
||||
} catch (IOException e) {
|
||||
throw new IllegalArgumentException(e);
|
||||
}
|
||||
}
|
||||
|
||||
public static SearchContextId decode(NamedWriteableRegistry namedWriteableRegistry, String id) {
|
||||
final ByteBuffer byteBuffer;
|
||||
try {
|
||||
byteBuffer = ByteBuffer.wrap(Base64.getUrlDecoder().decode(id));
|
||||
} catch (Exception e) {
|
||||
throw new IllegalArgumentException("invalid id: [" + id + "]", e);
|
||||
}
|
||||
try (StreamInput in = new NamedWriteableAwareStreamInput(new ByteBufferStreamInput(byteBuffer), namedWriteableRegistry)) {
|
||||
final Version version = Version.readVersion(in);
|
||||
in.setVersion(version);
|
||||
final Map<ShardId, SearchContextIdForNode> shards = in.readMap(ShardId::new, SearchContextIdForNode::new);
|
||||
final Map<String, AliasFilter> aliasFilters = in.readMap(StreamInput::readString, AliasFilter::new);
|
||||
if (in.available() > 0) {
|
||||
throw new IllegalArgumentException("Not all bytes were read");
|
||||
}
|
||||
return new SearchContextId(Collections.unmodifiableMap(shards), Collections.unmodifiableMap(aliasFilters));
|
||||
} catch (IOException e) {
|
||||
throw new IllegalArgumentException(e);
|
||||
}
|
||||
}
|
||||
|
||||
public String[] getActualIndices() {
|
||||
final Set<String> indices = new HashSet<>();
|
||||
for (Map.Entry<ShardId, SearchContextIdForNode> entry : shards().entrySet()) {
|
||||
final String indexName = entry.getKey().getIndexName();
|
||||
final String clusterAlias = entry.getValue().getClusterAlias();
|
||||
if (Strings.isEmpty(clusterAlias)) {
|
||||
indices.add(indexName);
|
||||
} else {
|
||||
indices.add(clusterAlias + RemoteClusterAware.REMOTE_CLUSTER_INDEX_SEPARATOR + indexName);
|
||||
}
|
||||
}
|
||||
return indices.toArray(new String[0]);
|
||||
}
|
||||
}
|
|
@ -20,17 +20,35 @@
|
|||
package org.elasticsearch.action.search;
|
||||
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
|
||||
class ScrollIdForNode {
|
||||
import java.io.IOException;
|
||||
|
||||
public final class SearchContextIdForNode implements Writeable {
|
||||
private final String node;
|
||||
private final SearchContextId contextId;
|
||||
private final ShardSearchContextId searchContextId;
|
||||
private final String clusterAlias;
|
||||
|
||||
ScrollIdForNode(@Nullable String clusterAlias, String node, SearchContextId contextId) {
|
||||
SearchContextIdForNode(@Nullable String clusterAlias, String node, ShardSearchContextId searchContextId) {
|
||||
this.node = node;
|
||||
this.clusterAlias = clusterAlias;
|
||||
this.contextId = contextId;
|
||||
this.searchContextId = searchContextId;
|
||||
}
|
||||
|
||||
SearchContextIdForNode(StreamInput in) throws IOException {
|
||||
this.node = in.readString();
|
||||
this.clusterAlias = in.readOptionalString();
|
||||
this.searchContextId = new ShardSearchContextId(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
out.writeString(node);
|
||||
out.writeOptionalString(clusterAlias);
|
||||
searchContextId.writeTo(out);
|
||||
}
|
||||
|
||||
public String getNode() {
|
||||
|
@ -42,15 +60,15 @@ class ScrollIdForNode {
|
|||
return clusterAlias;
|
||||
}
|
||||
|
||||
public SearchContextId getContextId() {
|
||||
return contextId;
|
||||
public ShardSearchContextId getSearchContextId() {
|
||||
return searchContextId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "ScrollIdForNode{" +
|
||||
return "SearchContextIdForNode{" +
|
||||
"node='" + node + '\'' +
|
||||
", scrollId=" + contextId +
|
||||
", seachContextId=" + searchContextId +
|
||||
", clusterAlias='" + clusterAlias + '\'' +
|
||||
'}';
|
||||
}
|
|
@ -25,10 +25,12 @@ import org.elasticsearch.cluster.ClusterState;
|
|||
import org.elasticsearch.cluster.routing.GroupShardsIterator;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||
import org.elasticsearch.search.dfs.AggregatedDfs;
|
||||
import org.elasticsearch.search.dfs.DfsSearchResult;
|
||||
import org.elasticsearch.search.internal.AliasFilter;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.Executor;
|
||||
|
@ -71,7 +73,10 @@ final class SearchDfsQueryThenFetchAsyncAction extends AbstractSearchAsyncAction
|
|||
|
||||
@Override
|
||||
protected SearchPhase getNextPhase(final SearchPhaseResults<DfsSearchResult> results, final SearchPhaseContext context) {
|
||||
return new DfsQueryPhase(results.getAtomicArray(), searchPhaseController, (queryResults) ->
|
||||
new FetchSearchPhase(queryResults, searchPhaseController, context, clusterState()), context, onPartialMergeFailure);
|
||||
final List<DfsSearchResult> dfsSearchResults = results.getAtomicArray().asList();
|
||||
final AggregatedDfs aggregatedDfs = searchPhaseController.aggregateDfs(dfsSearchResults);
|
||||
|
||||
return new DfsQueryPhase(dfsSearchResults, aggregatedDfs, searchPhaseController, (queryResults) ->
|
||||
new FetchSearchPhase(queryResults, searchPhaseController, aggregatedDfs, context), context, onPartialMergeFailure);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,9 +21,11 @@ package org.elasticsearch.action.search;
|
|||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.OriginalIndices;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.util.concurrent.AtomicArray;
|
||||
import org.elasticsearch.search.SearchPhaseResult;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.internal.InternalSearchResponse;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchRequest;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
|
||||
|
@ -57,10 +59,11 @@ interface SearchPhaseContext extends Executor {
|
|||
|
||||
/**
|
||||
* Builds and sends the final search response back to the user.
|
||||
*
|
||||
* @param internalSearchResponse the internal search response
|
||||
* @param scrollId an optional scroll ID if this search is a scroll search
|
||||
* @param queryResults the results of the query phase
|
||||
*/
|
||||
void sendSearchResponse(InternalSearchResponse internalSearchResponse, String scrollId);
|
||||
void sendSearchResponse(InternalSearchResponse internalSearchResponse, AtomicArray<SearchPhaseResult> queryResults);
|
||||
|
||||
/**
|
||||
* Notifies the top-level listener of the provided exception
|
||||
|
@ -101,7 +104,9 @@ interface SearchPhaseContext extends Executor {
|
|||
* @see org.elasticsearch.search.fetch.FetchSearchResult#getContextId()
|
||||
*
|
||||
*/
|
||||
default void sendReleaseSearchContext(SearchContextId contextId, Transport.Connection connection, OriginalIndices originalIndices) {
|
||||
default void sendReleaseSearchContext(ShardSearchContextId contextId,
|
||||
Transport.Connection connection,
|
||||
OriginalIndices originalIndices) {
|
||||
if (connection != null) {
|
||||
getSearchTransport().sendFreeContext(connection, contextId, originalIndices);
|
||||
}
|
||||
|
|
|
@ -110,7 +110,7 @@ class SearchQueryThenFetchAsyncAction extends AbstractSearchAsyncAction<SearchPh
|
|||
|
||||
@Override
|
||||
protected SearchPhase getNextPhase(final SearchPhaseResults<SearchPhaseResult> results, final SearchPhaseContext context) {
|
||||
return new FetchSearchPhase(results, searchPhaseController, context, clusterState());
|
||||
return new FetchSearchPhase(results, searchPhaseController, null, context);
|
||||
}
|
||||
|
||||
private ShardSearchRequest rewriteShardSearchRequest(ShardSearchRequest request) {
|
||||
|
|
|
@ -291,6 +291,17 @@ public class SearchRequest extends ActionRequest implements IndicesRequest.Repla
|
|||
validationException = source.aggregations().validate(validationException);
|
||||
}
|
||||
}
|
||||
if (pointInTimeBuilder() != null) {
|
||||
if (scroll) {
|
||||
validationException = addValidationError("using [point in time] is not allowed in a scroll context", validationException);
|
||||
}
|
||||
if (routing() != null) {
|
||||
validationException = addValidationError("[routing] cannot be used with point in time", validationException);
|
||||
}
|
||||
if (preference() != null) {
|
||||
validationException = addValidationError("[preference] cannot be used with point in time", validationException);
|
||||
}
|
||||
}
|
||||
return validationException;
|
||||
}
|
||||
|
||||
|
@ -474,6 +485,13 @@ public class SearchRequest extends ActionRequest implements IndicesRequest.Repla
|
|||
return source;
|
||||
}
|
||||
|
||||
public SearchSourceBuilder.PointInTimeBuilder pointInTimeBuilder() {
|
||||
if (source != null) {
|
||||
return source.pointInTimeBuilder();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* The tye of search to execute.
|
||||
*/
|
||||
|
|
|
@ -553,6 +553,17 @@ public class SearchRequestBuilder extends ActionRequestBuilder<SearchRequest, Se
|
|||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Specifies the search context that Elasticsearch should use to perform the query
|
||||
*
|
||||
* @param searchContextId the base64 encoded string of the search context id
|
||||
* @param keepAlive the extended time to live for the search context
|
||||
*/
|
||||
public SearchRequestBuilder setSearchContext(String searchContextId, TimeValue keepAlive) {
|
||||
sourceBuilder().pointInTimeBuilder(new SearchSourceBuilder.PointInTimeBuilder(searchContextId, keepAlive));
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
if (request.source() != null) {
|
||||
|
|
|
@ -62,6 +62,7 @@ import static org.elasticsearch.common.xcontent.XContentParserUtils.ensureExpect
|
|||
public class SearchResponse extends ActionResponse implements StatusToXContentObject {
|
||||
|
||||
private static final ParseField SCROLL_ID = new ParseField("_scroll_id");
|
||||
private static final ParseField POINT_IN_TIME_ID = new ParseField("pit_id");
|
||||
private static final ParseField TOOK = new ParseField("took");
|
||||
private static final ParseField TIMED_OUT = new ParseField("timed_out");
|
||||
private static final ParseField TERMINATED_EARLY = new ParseField("terminated_early");
|
||||
|
@ -69,6 +70,7 @@ public class SearchResponse extends ActionResponse implements StatusToXContentOb
|
|||
|
||||
private final SearchResponseSections internalResponse;
|
||||
private final String scrollId;
|
||||
private final String pointInTimeId;
|
||||
private final int totalShards;
|
||||
private final int successfulShards;
|
||||
private final int skippedShards;
|
||||
|
@ -98,12 +100,24 @@ public class SearchResponse extends ActionResponse implements StatusToXContentOb
|
|||
scrollId = in.readOptionalString();
|
||||
tookInMillis = in.readVLong();
|
||||
skippedShards = in.readVInt();
|
||||
if (in.getVersion().onOrAfter(Version.V_7_10_0)) {
|
||||
pointInTimeId = in.readOptionalString();
|
||||
} else {
|
||||
pointInTimeId = null;
|
||||
}
|
||||
}
|
||||
|
||||
public SearchResponse(SearchResponseSections internalResponse, String scrollId, int totalShards, int successfulShards,
|
||||
int skippedShards, long tookInMillis, ShardSearchFailure[] shardFailures, Clusters clusters) {
|
||||
this(internalResponse, scrollId, totalShards, successfulShards, skippedShards, tookInMillis, shardFailures, clusters, null);
|
||||
}
|
||||
|
||||
public SearchResponse(SearchResponseSections internalResponse, String scrollId, int totalShards, int successfulShards,
|
||||
int skippedShards, long tookInMillis, ShardSearchFailure[] shardFailures, Clusters clusters,
|
||||
String pointInTimeId) {
|
||||
this.internalResponse = internalResponse;
|
||||
this.scrollId = scrollId;
|
||||
this.pointInTimeId = pointInTimeId;
|
||||
this.clusters = clusters;
|
||||
this.totalShards = totalShards;
|
||||
this.successfulShards = successfulShards;
|
||||
|
@ -111,6 +125,8 @@ public class SearchResponse extends ActionResponse implements StatusToXContentOb
|
|||
this.tookInMillis = tookInMillis;
|
||||
this.shardFailures = shardFailures;
|
||||
assert skippedShards <= totalShards : "skipped: " + skippedShards + " total: " + totalShards;
|
||||
assert scrollId == null || pointInTimeId == null :
|
||||
"SearchResponse can't have both scrollId [" + scrollId + "] and searchContextId [" + pointInTimeId + "]";
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -212,6 +228,13 @@ public class SearchResponse extends ActionResponse implements StatusToXContentOb
|
|||
return scrollId;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the encoded string of the search context that the search request is used to executed
|
||||
*/
|
||||
public String pointInTimeId() {
|
||||
return pointInTimeId;
|
||||
}
|
||||
|
||||
/**
|
||||
* If profiling was enabled, this returns an object containing the profile results from
|
||||
* each shard. If profiling was not enabled, this will return null
|
||||
|
@ -244,6 +267,9 @@ public class SearchResponse extends ActionResponse implements StatusToXContentOb
|
|||
if (scrollId != null) {
|
||||
builder.field(SCROLL_ID.getPreferredName(), scrollId);
|
||||
}
|
||||
if (pointInTimeId != null) {
|
||||
builder.field(POINT_IN_TIME_ID.getPreferredName(), pointInTimeId);
|
||||
}
|
||||
builder.field(TOOK.getPreferredName(), tookInMillis);
|
||||
builder.field(TIMED_OUT.getPreferredName(), isTimedOut());
|
||||
if (isTerminatedEarly() != null) {
|
||||
|
@ -280,6 +306,7 @@ public class SearchResponse extends ActionResponse implements StatusToXContentOb
|
|||
int totalShards = -1;
|
||||
int skippedShards = 0; // 0 for BWC
|
||||
String scrollId = null;
|
||||
String searchContextId = null;
|
||||
List<ShardSearchFailure> failures = new ArrayList<>();
|
||||
Clusters clusters = Clusters.EMPTY;
|
||||
for (Token token = parser.nextToken(); token != Token.END_OBJECT; token = parser.nextToken()) {
|
||||
|
@ -288,6 +315,8 @@ public class SearchResponse extends ActionResponse implements StatusToXContentOb
|
|||
} else if (token.isValue()) {
|
||||
if (SCROLL_ID.match(currentFieldName, parser.getDeprecationHandler())) {
|
||||
scrollId = parser.text();
|
||||
} else if (POINT_IN_TIME_ID.match(currentFieldName, parser.getDeprecationHandler())) {
|
||||
searchContextId = parser.text();
|
||||
} else if (TOOK.match(currentFieldName, parser.getDeprecationHandler())) {
|
||||
tookInMillis = parser.longValue();
|
||||
} else if (TIMED_OUT.match(currentFieldName, parser.getDeprecationHandler())) {
|
||||
|
@ -366,7 +395,7 @@ public class SearchResponse extends ActionResponse implements StatusToXContentOb
|
|||
SearchResponseSections searchResponseSections = new SearchResponseSections(hits, aggs, suggest, timedOut, terminatedEarly,
|
||||
profile, numReducePhases);
|
||||
return new SearchResponse(searchResponseSections, scrollId, totalShards, successfulShards, skippedShards, tookInMillis,
|
||||
failures.toArray(ShardSearchFailure.EMPTY_ARRAY), clusters);
|
||||
failures.toArray(ShardSearchFailure.EMPTY_ARRAY), clusters, searchContextId);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -385,6 +414,9 @@ public class SearchResponse extends ActionResponse implements StatusToXContentOb
|
|||
out.writeOptionalString(scrollId);
|
||||
out.writeVLong(tookInMillis);
|
||||
out.writeVInt(skippedShards);
|
||||
if (out.getVersion().onOrAfter(Version.V_7_10_0)) {
|
||||
out.writeOptionalString(pointInTimeId);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -493,6 +525,6 @@ public class SearchResponse extends ActionResponse implements StatusToXContentOb
|
|||
InternalSearchResponse internalSearchResponse = new InternalSearchResponse(searchHits,
|
||||
InternalAggregations.EMPTY, null, null, false, null, 0);
|
||||
return new SearchResponse(internalSearchResponse, null, 0, 0, 0, tookInMillisSupplier.get(),
|
||||
ShardSearchFailure.EMPTY_ARRAY, clusters);
|
||||
ShardSearchFailure.EMPTY_ARRAY, clusters, null);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -203,7 +203,8 @@ final class SearchResponseMerger {
|
|||
InternalSearchResponse response = new InternalSearchResponse(mergedSearchHits, reducedAggs, suggest, profileShardResults,
|
||||
topDocsStats.timedOut, topDocsStats.terminatedEarly, numReducePhases);
|
||||
long tookInMillis = searchTimeProvider.buildTookInMillis();
|
||||
return new SearchResponse(response, null, totalShards, successfulShards, skippedShards, tookInMillis, shardFailures, clusters);
|
||||
return new SearchResponse(response, null, totalShards, successfulShards, skippedShards, tookInMillis, shardFailures,
|
||||
clusters, null);
|
||||
}
|
||||
|
||||
private static final Comparator<ShardSearchFailure> FAILURES_COMPARATOR = new Comparator<ShardSearchFailure>() {
|
||||
|
|
|
@ -31,7 +31,7 @@ import org.elasticsearch.search.SearchPhaseResult;
|
|||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.internal.InternalScrollSearchRequest;
|
||||
import org.elasticsearch.search.internal.InternalSearchResponse;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
import org.elasticsearch.transport.RemoteClusterService;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
|
||||
|
@ -104,7 +104,7 @@ abstract class SearchScrollAsyncAction<T extends SearchPhaseResult> implements R
|
|||
}
|
||||
|
||||
public final void run() {
|
||||
final ScrollIdForNode[] context = scrollId.getContext();
|
||||
final SearchContextIdForNode[] context = scrollId.getContext();
|
||||
if (context.length == 0) {
|
||||
listener.onFailure(new SearchPhaseExecutionException("query", "no nodes to search on", ShardSearchFailure.EMPTY_ARRAY));
|
||||
} else {
|
||||
|
@ -117,11 +117,11 @@ abstract class SearchScrollAsyncAction<T extends SearchPhaseResult> implements R
|
|||
* This method collects nodes from the remote clusters asynchronously if any of the scroll IDs references a remote cluster.
|
||||
* Otherwise the action listener will be invoked immediately with a function based on the given discovery nodes.
|
||||
*/
|
||||
static void collectNodesAndRun(final Iterable<ScrollIdForNode> scrollIds, DiscoveryNodes nodes,
|
||||
static void collectNodesAndRun(final Iterable<SearchContextIdForNode> scrollIds, DiscoveryNodes nodes,
|
||||
SearchTransportService searchTransportService,
|
||||
ActionListener<BiFunction<String, String, DiscoveryNode>> listener) {
|
||||
Set<String> clusters = new HashSet<>();
|
||||
for (ScrollIdForNode target : scrollIds) {
|
||||
for (SearchContextIdForNode target : scrollIds) {
|
||||
if (target.getClusterAlias() != null) {
|
||||
clusters.add(target.getClusterAlias());
|
||||
}
|
||||
|
@ -135,10 +135,10 @@ abstract class SearchScrollAsyncAction<T extends SearchPhaseResult> implements R
|
|||
}
|
||||
}
|
||||
|
||||
private void run(BiFunction<String, String, DiscoveryNode> clusterNodeLookup, final ScrollIdForNode[] context) {
|
||||
private void run(BiFunction<String, String, DiscoveryNode> clusterNodeLookup, final SearchContextIdForNode[] context) {
|
||||
final CountDown counter = new CountDown(scrollId.getContext().length);
|
||||
for (int i = 0; i < context.length; i++) {
|
||||
ScrollIdForNode target = context[i];
|
||||
SearchContextIdForNode target = context[i];
|
||||
final int shardIndex = i;
|
||||
final Transport.Connection connection;
|
||||
try {
|
||||
|
@ -148,11 +148,11 @@ abstract class SearchScrollAsyncAction<T extends SearchPhaseResult> implements R
|
|||
}
|
||||
connection = getConnection(target.getClusterAlias(), node);
|
||||
} catch (Exception ex) {
|
||||
onShardFailure("query", counter, target.getContextId(),
|
||||
onShardFailure("query", counter, target.getSearchContextId(),
|
||||
ex, null, () -> SearchScrollAsyncAction.this.moveToNextPhase(clusterNodeLookup));
|
||||
continue;
|
||||
}
|
||||
final InternalScrollSearchRequest internalRequest = internalScrollSearchRequest(target.getContextId(), request);
|
||||
final InternalScrollSearchRequest internalRequest = internalScrollSearchRequest(target.getSearchContextId(), request);
|
||||
// we can't create a SearchShardTarget here since we don't know the index and shard ID we are talking to
|
||||
// we only know the node and the search context ID. Yet, the response will contain the SearchShardTarget
|
||||
// from the target node instead...that's why we pass null here
|
||||
|
@ -192,7 +192,7 @@ abstract class SearchScrollAsyncAction<T extends SearchPhaseResult> implements R
|
|||
|
||||
@Override
|
||||
public void onFailure(Exception t) {
|
||||
onShardFailure("query", counter, target.getContextId(), t, null,
|
||||
onShardFailure("query", counter, target.getSearchContextId(), t, null,
|
||||
() -> SearchScrollAsyncAction.this.moveToNextPhase(clusterNodeLookup));
|
||||
}
|
||||
};
|
||||
|
@ -242,13 +242,13 @@ abstract class SearchScrollAsyncAction<T extends SearchPhaseResult> implements R
|
|||
scrollId = request.scrollId();
|
||||
}
|
||||
listener.onResponse(new SearchResponse(internalResponse, scrollId, this.scrollId.getContext().length, successfulOps.get(),
|
||||
0, buildTookInMillis(), buildShardFailures(), SearchResponse.Clusters.EMPTY));
|
||||
0, buildTookInMillis(), buildShardFailures(), SearchResponse.Clusters.EMPTY, null));
|
||||
} catch (Exception e) {
|
||||
listener.onFailure(new ReduceSearchPhaseException("fetch", "inner finish failed", e, buildShardFailures()));
|
||||
}
|
||||
}
|
||||
|
||||
protected void onShardFailure(String phaseName, final CountDown counter, final SearchContextId searchId, Exception failure,
|
||||
protected void onShardFailure(String phaseName, final CountDown counter, final ShardSearchContextId searchId, Exception failure,
|
||||
@Nullable SearchShardTarget searchShardTarget,
|
||||
Supplier<SearchPhase> nextPhaseSupplier) {
|
||||
if (logger.isDebugEnabled()) {
|
||||
|
|
|
@ -24,8 +24,10 @@ import org.elasticsearch.cluster.routing.PlainShardIterator;
|
|||
import org.elasticsearch.cluster.routing.ShardIterator;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
@ -42,6 +44,9 @@ public final class SearchShardIterator extends PlainShardIterator {
|
|||
private final String clusterAlias;
|
||||
private boolean skip = false;
|
||||
|
||||
private final ShardSearchContextId searchContextId;
|
||||
private final TimeValue searchContextKeepAlive;
|
||||
|
||||
/**
|
||||
* Creates a {@link PlainShardIterator} instance that iterates over a subset of the given shards
|
||||
* this the a given <code>shardId</code>.
|
||||
|
@ -52,9 +57,18 @@ public final class SearchShardIterator extends PlainShardIterator {
|
|||
* @param originalIndices the indices that the search request originally related to (before any rewriting happened)
|
||||
*/
|
||||
public SearchShardIterator(@Nullable String clusterAlias, ShardId shardId, List<ShardRouting> shards, OriginalIndices originalIndices) {
|
||||
this(clusterAlias, shardId, shards, originalIndices, null, null);
|
||||
}
|
||||
|
||||
public SearchShardIterator(@Nullable String clusterAlias, ShardId shardId,
|
||||
List<ShardRouting> shards, OriginalIndices originalIndices,
|
||||
ShardSearchContextId searchContextId, TimeValue searchContextKeepAlive) {
|
||||
super(shardId, shards);
|
||||
this.originalIndices = originalIndices;
|
||||
this.clusterAlias = clusterAlias;
|
||||
this.searchContextId = searchContextId;
|
||||
this.searchContextKeepAlive = searchContextKeepAlive;
|
||||
assert (searchContextId == null) == (searchContextKeepAlive == null);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -80,6 +94,17 @@ public final class SearchShardIterator extends PlainShardIterator {
|
|||
return new SearchShardTarget(nodeId, shardId(), clusterAlias, originalIndices);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a non-null value if this request should use a specific search context instead of the latest one.
|
||||
*/
|
||||
ShardSearchContextId getSearchContextId() {
|
||||
return searchContextId;
|
||||
}
|
||||
|
||||
TimeValue getSearchContextKeepAlive() {
|
||||
return searchContextKeepAlive;
|
||||
}
|
||||
|
||||
/**
|
||||
* Reset the iterator and mark it as skippable
|
||||
* @see #skip()
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package org.elasticsearch.action.search;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.ActionListenerResponseHandler;
|
||||
import org.elasticsearch.action.IndicesRequest;
|
||||
|
@ -40,7 +41,7 @@ import org.elasticsearch.search.fetch.ScrollQueryFetchSearchResult;
|
|||
import org.elasticsearch.search.fetch.ShardFetchRequest;
|
||||
import org.elasticsearch.search.fetch.ShardFetchSearchRequest;
|
||||
import org.elasticsearch.search.internal.InternalScrollSearchRequest;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchRequest;
|
||||
import org.elasticsearch.search.query.QuerySearchRequest;
|
||||
import org.elasticsearch.search.query.QuerySearchResult;
|
||||
|
@ -89,7 +90,7 @@ public class SearchTransportService {
|
|||
this.responseWrapper = responseWrapper;
|
||||
}
|
||||
|
||||
public void sendFreeContext(Transport.Connection connection, final SearchContextId contextId, OriginalIndices originalIndices) {
|
||||
public void sendFreeContext(Transport.Connection connection, final ShardSearchContextId contextId, OriginalIndices originalIndices) {
|
||||
transportService.sendRequest(connection, FREE_CONTEXT_ACTION_NAME, new SearchFreeContextRequest(originalIndices, contextId),
|
||||
TransportRequestOptions.EMPTY, new ActionListenerResponseHandler<>(new ActionListener<SearchFreeContextResponse>() {
|
||||
@Override
|
||||
|
@ -104,7 +105,7 @@ public class SearchTransportService {
|
|||
}, SearchFreeContextResponse::new));
|
||||
}
|
||||
|
||||
public void sendFreeContext(Transport.Connection connection, SearchContextId contextId,
|
||||
public void sendFreeContext(Transport.Connection connection, ShardSearchContextId contextId,
|
||||
ActionListener<SearchFreeContextResponse> listener) {
|
||||
transportService.sendRequest(connection, FREE_CONTEXT_SCROLL_ACTION_NAME, new ScrollFreeContextRequest(contextId),
|
||||
TransportRequestOptions.EMPTY, new ActionListenerResponseHandler<>(listener, SearchFreeContextResponse::new));
|
||||
|
@ -197,15 +198,15 @@ public class SearchTransportService {
|
|||
}
|
||||
|
||||
static class ScrollFreeContextRequest extends TransportRequest {
|
||||
private SearchContextId contextId;
|
||||
private ShardSearchContextId contextId;
|
||||
|
||||
ScrollFreeContextRequest(SearchContextId contextId) {
|
||||
ScrollFreeContextRequest(ShardSearchContextId contextId) {
|
||||
this.contextId = Objects.requireNonNull(contextId);
|
||||
}
|
||||
|
||||
ScrollFreeContextRequest(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
contextId = new SearchContextId(in);
|
||||
contextId = new ShardSearchContextId(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -214,7 +215,7 @@ public class SearchTransportService {
|
|||
contextId.writeTo(out);
|
||||
}
|
||||
|
||||
public SearchContextId id() {
|
||||
public ShardSearchContextId id() {
|
||||
return this.contextId;
|
||||
}
|
||||
|
||||
|
@ -223,7 +224,7 @@ public class SearchTransportService {
|
|||
static class SearchFreeContextRequest extends ScrollFreeContextRequest implements IndicesRequest {
|
||||
private OriginalIndices originalIndices;
|
||||
|
||||
SearchFreeContextRequest(OriginalIndices originalIndices, SearchContextId id) {
|
||||
SearchFreeContextRequest(OriginalIndices originalIndices, ShardSearchContextId id) {
|
||||
super(id);
|
||||
this.originalIndices = originalIndices;
|
||||
}
|
||||
|
@ -279,16 +280,20 @@ public class SearchTransportService {
|
|||
}
|
||||
}
|
||||
|
||||
static boolean keepStatesInContext(Version version) {
|
||||
return version.before(Version.V_7_10_0);
|
||||
}
|
||||
|
||||
public static void registerRequestHandler(TransportService transportService, SearchService searchService) {
|
||||
transportService.registerRequestHandler(FREE_CONTEXT_SCROLL_ACTION_NAME, ThreadPool.Names.SAME, ScrollFreeContextRequest::new,
|
||||
(request, channel, task) -> {
|
||||
boolean freed = searchService.freeContext(request.id());
|
||||
boolean freed = searchService.freeReaderContext(request.id());
|
||||
channel.sendResponse(new SearchFreeContextResponse(freed));
|
||||
});
|
||||
TransportActionProxy.registerProxyAction(transportService, FREE_CONTEXT_SCROLL_ACTION_NAME, SearchFreeContextResponse::new);
|
||||
transportService.registerRequestHandler(FREE_CONTEXT_ACTION_NAME, ThreadPool.Names.SAME, SearchFreeContextRequest::new,
|
||||
(request, channel, task) -> {
|
||||
boolean freed = searchService.freeContext(request.id());
|
||||
boolean freed = searchService.freeReaderContext(request.id());
|
||||
channel.sendResponse(new SearchFreeContextResponse(freed));
|
||||
});
|
||||
TransportActionProxy.registerProxyAction(transportService, FREE_CONTEXT_ACTION_NAME, SearchFreeContextResponse::new);
|
||||
|
@ -303,7 +308,7 @@ public class SearchTransportService {
|
|||
|
||||
transportService.registerRequestHandler(DFS_ACTION_NAME, ThreadPool.Names.SAME, ShardSearchRequest::new,
|
||||
(request, channel, task) ->
|
||||
searchService.executeDfsPhase(request, (SearchShardTask) task,
|
||||
searchService.executeDfsPhase(request, keepStatesInContext(channel.getVersion()), (SearchShardTask) task,
|
||||
new ChannelActionListener<>(channel, DFS_ACTION_NAME, request))
|
||||
);
|
||||
|
||||
|
@ -311,7 +316,7 @@ public class SearchTransportService {
|
|||
|
||||
transportService.registerRequestHandler(QUERY_ACTION_NAME, ThreadPool.Names.SAME, ShardSearchRequest::new,
|
||||
(request, channel, task) -> {
|
||||
searchService.executeQueryPhase(request, (SearchShardTask) task,
|
||||
searchService.executeQueryPhase(request, keepStatesInContext(channel.getVersion()), (SearchShardTask) task,
|
||||
new ChannelActionListener<>(channel, QUERY_ACTION_NAME, request));
|
||||
});
|
||||
TransportActionProxy.registerProxyActionWithDynamicResponseType(transportService, QUERY_ACTION_NAME,
|
||||
|
@ -368,7 +373,7 @@ public class SearchTransportService {
|
|||
* @param node the node to resolve
|
||||
* @return a connection to the given node belonging to the cluster with the provided alias.
|
||||
*/
|
||||
Transport.Connection getConnection(@Nullable String clusterAlias, DiscoveryNode node) {
|
||||
public Transport.Connection getConnection(@Nullable String clusterAlias, DiscoveryNode node) {
|
||||
if (clusterAlias == null) {
|
||||
return transportService.getConnection(node);
|
||||
} else {
|
||||
|
|
|
@ -24,6 +24,7 @@ import org.elasticsearch.action.support.ActionFilters;
|
|||
import org.elasticsearch.action.support.HandledTransportAction;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
|
||||
|
@ -31,18 +32,21 @@ public class TransportClearScrollAction extends HandledTransportAction<ClearScro
|
|||
|
||||
private final ClusterService clusterService;
|
||||
private final SearchTransportService searchTransportService;
|
||||
private final NamedWriteableRegistry namedWriteableRegistry;
|
||||
|
||||
@Inject
|
||||
public TransportClearScrollAction(TransportService transportService, ClusterService clusterService, ActionFilters actionFilters,
|
||||
SearchTransportService searchTransportService) {
|
||||
SearchTransportService searchTransportService, NamedWriteableRegistry namedWriteableRegistry) {
|
||||
super(ClearScrollAction.NAME, transportService, actionFilters, ClearScrollRequest::new);
|
||||
this.clusterService = clusterService;
|
||||
this.searchTransportService = searchTransportService;
|
||||
this.namedWriteableRegistry = namedWriteableRegistry;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doExecute(Task task, ClearScrollRequest request, final ActionListener<ClearScrollResponse> listener) {
|
||||
Runnable runnable = new ClearScrollController(request, listener, clusterService.state().nodes(), logger, searchTransportService);
|
||||
Runnable runnable = new ClearScrollController(
|
||||
request, listener, clusterService.state().nodes(), logger, searchTransportService);
|
||||
runnable.run();
|
||||
}
|
||||
|
||||
|
|
|
@ -34,23 +34,30 @@ import org.elasticsearch.client.node.NodeClient;
|
|||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.block.ClusterBlockException;
|
||||
import org.elasticsearch.cluster.block.ClusterBlockLevel;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetadata;
|
||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||
import org.elasticsearch.cluster.routing.GroupShardsIterator;
|
||||
import org.elasticsearch.cluster.routing.OperationRouting;
|
||||
import org.elasticsearch.cluster.routing.ShardIterator;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.common.settings.Setting;
|
||||
import org.elasticsearch.common.settings.Setting.Property;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.concurrent.AtomicArray;
|
||||
import org.elasticsearch.common.util.concurrent.CountDown;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.query.Rewriteable;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.search.SearchPhaseResult;
|
||||
import org.elasticsearch.search.SearchService;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregations;
|
||||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||
|
@ -72,6 +79,7 @@ import java.util.ArrayList;
|
|||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
@ -84,6 +92,8 @@ import java.util.function.BiConsumer;
|
|||
import java.util.function.BiFunction;
|
||||
import java.util.function.Function;
|
||||
import java.util.function.LongSupplier;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.StreamSupport;
|
||||
|
||||
import static org.elasticsearch.action.admin.cluster.node.tasks.get.GetTaskAction.TASKS_ORIGIN;
|
||||
import static org.elasticsearch.action.search.SearchType.DFS_QUERY_THEN_FETCH;
|
||||
|
@ -104,12 +114,19 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
private final SearchPhaseController searchPhaseController;
|
||||
private final SearchService searchService;
|
||||
private final IndexNameExpressionResolver indexNameExpressionResolver;
|
||||
private final NamedWriteableRegistry namedWriteableRegistry;
|
||||
|
||||
@Inject
|
||||
public TransportSearchAction(NodeClient client, ThreadPool threadPool, TransportService transportService,
|
||||
SearchService searchService, SearchTransportService searchTransportService,
|
||||
SearchPhaseController searchPhaseController, ClusterService clusterService,
|
||||
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) {
|
||||
public TransportSearchAction(NodeClient client,
|
||||
ThreadPool threadPool,
|
||||
TransportService transportService,
|
||||
SearchService searchService,
|
||||
SearchTransportService searchTransportService,
|
||||
SearchPhaseController searchPhaseController,
|
||||
ClusterService clusterService,
|
||||
ActionFilters actionFilters,
|
||||
IndexNameExpressionResolver indexNameExpressionResolver,
|
||||
NamedWriteableRegistry namedWriteableRegistry) {
|
||||
super(SearchAction.NAME, transportService, actionFilters, (Writeable.Reader<SearchRequest>) SearchRequest::new);
|
||||
this.client = client;
|
||||
this.threadPool = threadPool;
|
||||
|
@ -120,6 +137,7 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
this.clusterService = clusterService;
|
||||
this.searchService = searchService;
|
||||
this.indexNameExpressionResolver = indexNameExpressionResolver;
|
||||
this.namedWriteableRegistry = namedWriteableRegistry;
|
||||
}
|
||||
|
||||
private Map<String, AliasFilter> buildPerIndexAliasFilter(SearchRequest request, ClusterState clusterState,
|
||||
|
@ -203,6 +221,59 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
|
||||
@Override
|
||||
protected void doExecute(Task task, SearchRequest searchRequest, ActionListener<SearchResponse> listener) {
|
||||
executeRequest(task, searchRequest, this::searchAsyncAction, listener);
|
||||
}
|
||||
|
||||
public interface SinglePhaseSearchAction {
|
||||
void executeOnShardTarget(SearchTask searchTask, SearchShardTarget target, Transport.Connection connection,
|
||||
ActionListener<SearchPhaseResult> listener);
|
||||
}
|
||||
|
||||
public void executeRequest(Task task, SearchRequest searchRequest, String actionName,
|
||||
boolean includeSearchContext, SinglePhaseSearchAction phaseSearchAction,
|
||||
ActionListener<SearchResponse> listener) {
|
||||
executeRequest(task, searchRequest, new SearchAsyncActionProvider() {
|
||||
@Override
|
||||
public AbstractSearchAsyncAction<? extends SearchPhaseResult> asyncSearchAction(
|
||||
SearchTask task, SearchRequest searchRequest, Executor executor, GroupShardsIterator<SearchShardIterator> shardsIts,
|
||||
SearchTimeProvider timeProvider, BiFunction<String, String, Transport.Connection> connectionLookup,
|
||||
ClusterState clusterState, Map<String, AliasFilter> aliasFilter,
|
||||
Map<String, Float> concreteIndexBoosts, Map<String, Set<String>> indexRoutings,
|
||||
ActionListener<SearchResponse> listener, boolean preFilter, ThreadPool threadPool, SearchResponse.Clusters clusters) {
|
||||
return new AbstractSearchAsyncAction<SearchPhaseResult>(
|
||||
actionName, logger, searchTransportService, connectionLookup, aliasFilter, concreteIndexBoosts,
|
||||
indexRoutings, executor, searchRequest, listener, shardsIts, timeProvider, clusterState, task,
|
||||
new ArraySearchPhaseResults<>(shardsIts.size()), 1, clusters) {
|
||||
@Override
|
||||
protected void executePhaseOnShard(SearchShardIterator shardIt, ShardRouting shard,
|
||||
SearchActionListener<SearchPhaseResult> listener) {
|
||||
final Transport.Connection connection = getConnection(shardIt.getClusterAlias(), shard.currentNodeId());
|
||||
final SearchShardTarget searchShardTarget = shardIt.newSearchShardTarget(shard.currentNodeId());
|
||||
phaseSearchAction.executeOnShardTarget(task, searchShardTarget, connection, listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected SearchPhase getNextPhase(SearchPhaseResults<SearchPhaseResult> results, SearchPhaseContext context) {
|
||||
return new SearchPhase(getName()) {
|
||||
@Override
|
||||
public void run() {
|
||||
final AtomicArray<SearchPhaseResult> atomicArray = results.getAtomicArray();
|
||||
sendSearchResponse(InternalSearchResponse.empty(), atomicArray);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
boolean includeSearchContextInResponse() {
|
||||
return includeSearchContext;
|
||||
}
|
||||
};
|
||||
}
|
||||
}, listener);
|
||||
}
|
||||
|
||||
private void executeRequest(Task task, SearchRequest searchRequest,
|
||||
SearchAsyncActionProvider searchAsyncActionProvider, ActionListener<SearchResponse> listener) {
|
||||
final long relativeStartNanos = System.nanoTime();
|
||||
final SearchTimeProvider timeProvider =
|
||||
new SearchTimeProvider(searchRequest.getOrCreateAbsoluteStartMillis(), relativeStartNanos, System::nanoTime);
|
||||
|
@ -213,17 +284,27 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
searchRequest.source(source);
|
||||
}
|
||||
final ClusterState clusterState = clusterService.state();
|
||||
final Map<String, OriginalIndices> remoteClusterIndices = remoteClusterService.groupIndices(searchRequest.indicesOptions(),
|
||||
searchRequest.indices(), idx -> indexNameExpressionResolver.hasIndexAbstraction(idx, clusterState));
|
||||
final SearchContextId searchContext;
|
||||
final Map<String, OriginalIndices> remoteClusterIndices;
|
||||
if (searchRequest.pointInTimeBuilder() != null) {
|
||||
searchContext = SearchContextId.decode(namedWriteableRegistry, searchRequest.pointInTimeBuilder().getId());
|
||||
remoteClusterIndices = getIndicesFromSearchContexts(searchContext, searchRequest.indicesOptions());
|
||||
} else {
|
||||
searchContext = null;
|
||||
remoteClusterIndices = remoteClusterService.groupIndices(searchRequest.indicesOptions(),
|
||||
searchRequest.indices(), idx -> indexNameExpressionResolver.hasIndexAbstraction(idx, clusterState));
|
||||
}
|
||||
OriginalIndices localIndices = remoteClusterIndices.remove(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY);
|
||||
if (remoteClusterIndices.isEmpty()) {
|
||||
executeLocalSearch(task, timeProvider, searchRequest, localIndices, clusterState, listener);
|
||||
executeLocalSearch(
|
||||
task, timeProvider, searchRequest, localIndices, clusterState, listener, searchContext, searchAsyncActionProvider);
|
||||
} else {
|
||||
if (shouldMinimizeRoundtrips(searchRequest)) {
|
||||
ccsRemoteReduce(searchRequest, localIndices, remoteClusterIndices, timeProvider,
|
||||
searchService.aggReduceContextBuilder(searchRequest),
|
||||
remoteClusterService, threadPool, listener,
|
||||
(r, l) -> executeLocalSearch(task, timeProvider, r, localIndices, clusterState, l));
|
||||
searchService.aggReduceContextBuilder(searchRequest),
|
||||
remoteClusterService, threadPool, listener,
|
||||
(r, l) -> executeLocalSearch(
|
||||
task, timeProvider, r, localIndices, clusterState, l, searchContext, searchAsyncActionProvider));
|
||||
} else {
|
||||
AtomicInteger skippedClusters = new AtomicInteger(0);
|
||||
collectSearchShards(searchRequest.indicesOptions(), searchRequest.preference(), searchRequest.routing(),
|
||||
|
@ -237,9 +318,10 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
int localClusters = localIndices == null ? 0 : 1;
|
||||
int totalClusters = remoteClusterIndices.size() + localClusters;
|
||||
int successfulClusters = searchShardsResponses.size() + localClusters;
|
||||
executeSearch((SearchTask) task, timeProvider, searchRequest, localIndices,
|
||||
remoteShardIterators, clusterNodeLookup, clusterState, remoteAliasFilters, listener,
|
||||
new SearchResponse.Clusters(totalClusters, successfulClusters, skippedClusters.get()));
|
||||
executeSearch((SearchTask) task, timeProvider, searchRequest, localIndices, remoteShardIterators,
|
||||
clusterNodeLookup, clusterState, remoteAliasFilters, listener,
|
||||
new SearchResponse.Clusters(totalClusters, successfulClusters, skippedClusters.get()),
|
||||
searchContext, searchAsyncActionProvider);
|
||||
},
|
||||
listener::onFailure));
|
||||
}
|
||||
|
@ -260,6 +342,9 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
if (searchRequest.scroll() != null) {
|
||||
return false;
|
||||
}
|
||||
if (searchRequest.pointInTimeBuilder() != null) {
|
||||
return false;
|
||||
}
|
||||
if (searchRequest.searchType() == DFS_QUERY_THEN_FETCH) {
|
||||
return false;
|
||||
}
|
||||
|
@ -294,7 +379,8 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
searchResponse.isTimedOut(), searchResponse.isTerminatedEarly(), searchResponse.getNumReducePhases());
|
||||
listener.onResponse(new SearchResponse(internalSearchResponse, searchResponse.getScrollId(),
|
||||
searchResponse.getTotalShards(), searchResponse.getSuccessfulShards(), searchResponse.getSkippedShards(),
|
||||
timeProvider.buildTookInMillis(), searchResponse.getShardFailures(), new SearchResponse.Clusters(1, 1, 0)));
|
||||
timeProvider.buildTookInMillis(), searchResponse.getShardFailures(), new SearchResponse.Clusters(1, 1, 0),
|
||||
searchResponse.pointInTimeId()));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -406,9 +492,12 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
}
|
||||
|
||||
private void executeLocalSearch(Task task, SearchTimeProvider timeProvider, SearchRequest searchRequest, OriginalIndices localIndices,
|
||||
ClusterState clusterState, ActionListener<SearchResponse> listener) {
|
||||
ClusterState clusterState, ActionListener<SearchResponse> listener,
|
||||
SearchContextId searchContext,
|
||||
SearchAsyncActionProvider searchAsyncActionProvider) {
|
||||
executeSearch((SearchTask)task, timeProvider, searchRequest, localIndices, Collections.emptyList(),
|
||||
(clusterName, nodeId) -> null, clusterState, Collections.emptyMap(), listener, SearchResponse.Clusters.EMPTY);
|
||||
(clusterName, nodeId) -> null, clusterState, Collections.emptyMap(), listener, SearchResponse.Clusters.EMPTY,
|
||||
searchContext, searchAsyncActionProvider);
|
||||
}
|
||||
|
||||
static BiFunction<String, String, DiscoveryNode> processRemoteShards(Map<String, ClusterSearchShardsResponse> searchShardsResponses,
|
||||
|
@ -470,26 +559,52 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
OriginalIndices localIndices, List<SearchShardIterator> remoteShardIterators,
|
||||
BiFunction<String, String, DiscoveryNode> remoteConnections, ClusterState clusterState,
|
||||
Map<String, AliasFilter> remoteAliasMap, ActionListener<SearchResponse> listener,
|
||||
SearchResponse.Clusters clusters) {
|
||||
SearchResponse.Clusters clusters, @Nullable SearchContextId searchContext,
|
||||
SearchAsyncActionProvider searchAsyncActionProvider) {
|
||||
|
||||
clusterState.blocks().globalBlockedRaiseException(ClusterBlockLevel.READ);
|
||||
|
||||
// TODO: I think startTime() should become part of ActionRequest and that should be used both for index name
|
||||
// date math expressions and $now in scripts. This way all apis will deal with now in the same way instead
|
||||
// of just for the _search api
|
||||
final Index[] indices = resolveLocalIndices(localIndices, clusterState, timeProvider);
|
||||
Map<String, AliasFilter> aliasFilter = buildPerIndexAliasFilter(searchRequest, clusterState, indices, remoteAliasMap);
|
||||
Map<String, Set<String>> routingMap = indexNameExpressionResolver.resolveSearchRouting(clusterState, searchRequest.routing(),
|
||||
searchRequest.indices());
|
||||
routingMap = routingMap == null ? Collections.emptyMap() : Collections.unmodifiableMap(routingMap);
|
||||
String[] concreteIndices = new String[indices.length];
|
||||
for (int i = 0; i < indices.length; i++) {
|
||||
concreteIndices[i] = indices[i].getName();
|
||||
}
|
||||
Map<String, Long> nodeSearchCounts = searchTransportService.getPendingSearchRequests();
|
||||
GroupShardsIterator<ShardIterator> localShardsIterator = clusterService.operationRouting().searchShards(clusterState,
|
||||
final List<SearchShardIterator> localShardIterators;
|
||||
final Map<String, AliasFilter> aliasFilter;
|
||||
final Map<String, Set<String>> indexRoutings;
|
||||
final Executor asyncSearchExecutor;
|
||||
|
||||
boolean preFilterSearchShards;
|
||||
if (searchContext != null) {
|
||||
assert searchRequest.pointInTimeBuilder() != null;
|
||||
aliasFilter = searchContext.aliasFilter();
|
||||
indexRoutings = Collections.emptyMap();
|
||||
asyncSearchExecutor = asyncSearchExecutor(localIndices.indices(), clusterState);
|
||||
localShardIterators = getSearchShardsFromSearchContexts(clusterState, localIndices, searchRequest.getLocalClusterAlias(),
|
||||
searchContext, searchRequest.pointInTimeBuilder().getKeepAlive());
|
||||
preFilterSearchShards = shouldPreFilterSearchShards(clusterState, searchRequest, localIndices.indices(),
|
||||
localShardIterators.size() + remoteShardIterators.size());
|
||||
} else {
|
||||
final Index[] indices = resolveLocalIndices(localIndices, clusterState, timeProvider);
|
||||
Map<String, Set<String>> routingMap = indexNameExpressionResolver.resolveSearchRouting(clusterState, searchRequest.routing(),
|
||||
searchRequest.indices());
|
||||
routingMap = routingMap == null ? Collections.emptyMap() : Collections.unmodifiableMap(routingMap);
|
||||
final String[] concreteIndices = new String[indices.length];
|
||||
for (int i = 0; i < indices.length; i++) {
|
||||
concreteIndices[i] = indices[i].getName();
|
||||
}
|
||||
asyncSearchExecutor = asyncSearchExecutor(concreteIndices, clusterState);
|
||||
Map<String, Long> nodeSearchCounts = searchTransportService.getPendingSearchRequests();
|
||||
GroupShardsIterator<ShardIterator> localShardRoutings = clusterService.operationRouting().searchShards(clusterState,
|
||||
concreteIndices, routingMap, searchRequest.preference(), searchService.getResponseCollectorService(), nodeSearchCounts);
|
||||
GroupShardsIterator<SearchShardIterator> shardIterators = mergeShardsIterators(localShardsIterator, localIndices,
|
||||
searchRequest.getLocalClusterAlias(), remoteShardIterators);
|
||||
localShardIterators = StreamSupport.stream(localShardRoutings.spliterator(), false)
|
||||
.map(it -> new SearchShardIterator(
|
||||
searchRequest.getLocalClusterAlias(), it.shardId(), it.getShardRoutings(), localIndices, null, null))
|
||||
.collect(Collectors.toList());
|
||||
aliasFilter = buildPerIndexAliasFilter(searchRequest, clusterState, indices, remoteAliasMap);
|
||||
indexRoutings = routingMap;
|
||||
preFilterSearchShards = shouldPreFilterSearchShards(clusterState, searchRequest, concreteIndices,
|
||||
localShardIterators.size() + remoteShardIterators.size());
|
||||
}
|
||||
final GroupShardsIterator<SearchShardIterator> shardIterators = mergeShardsIterators(localShardIterators, remoteShardIterators);
|
||||
|
||||
failIfOverShardCountLimit(clusterService, shardIterators.size());
|
||||
|
||||
|
@ -501,7 +616,7 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
searchRequest.searchType(QUERY_THEN_FETCH);
|
||||
}
|
||||
if (searchRequest.allowPartialSearchResults() == null) {
|
||||
// No user preference defined in search request - apply cluster service default
|
||||
// No user preference defined in search request - apply cluster service default
|
||||
searchRequest.allowPartialSearchResults(searchService.defaultAllowPartialSearchResults());
|
||||
}
|
||||
if (searchRequest.isSuggestOnly()) {
|
||||
|
@ -514,19 +629,21 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
break;
|
||||
}
|
||||
}
|
||||
|
||||
final DiscoveryNodes nodes = clusterState.nodes();
|
||||
BiFunction<String, String, Transport.Connection> connectionLookup = buildConnectionLookup(searchRequest.getLocalClusterAlias(),
|
||||
nodes::get, remoteConnections, searchTransportService::getConnection);
|
||||
boolean preFilterSearchShards = shouldPreFilterSearchShards(clusterState, searchRequest, indices, shardIterators.size());
|
||||
final Executor asyncSearchExecutor = asyncSearchExecutor(indices, clusterState);
|
||||
searchAsyncAction(task, searchRequest, asyncSearchExecutor, shardIterators, timeProvider, connectionLookup, clusterState,
|
||||
Collections.unmodifiableMap(aliasFilter), concreteIndexBoosts, routingMap, listener, preFilterSearchShards, clusters).start();
|
||||
searchAsyncActionProvider.asyncSearchAction(
|
||||
task, searchRequest, asyncSearchExecutor, shardIterators, timeProvider, connectionLookup, clusterState,
|
||||
Collections.unmodifiableMap(aliasFilter), concreteIndexBoosts, indexRoutings, listener,
|
||||
preFilterSearchShards, threadPool, clusters).start();
|
||||
}
|
||||
|
||||
Executor asyncSearchExecutor(final Index[] indices, final ClusterState clusterState) {
|
||||
final boolean onlySystemIndices =
|
||||
Arrays.stream(indices).allMatch(index -> clusterState.metadata().index(index.getName()).isSystem());
|
||||
Executor asyncSearchExecutor(final String[] indices, final ClusterState clusterState) {
|
||||
final boolean onlySystemIndices = Arrays.stream(indices)
|
||||
.allMatch(index -> {
|
||||
final IndexMetadata indexMetadata = clusterState.metadata().index(index);
|
||||
return indexMetadata != null && indexMetadata.isSystem();
|
||||
});
|
||||
return onlySystemIndices ? threadPool.executor(ThreadPool.Names.SYSTEM_READ) : threadPool.executor(ThreadPool.Names.SEARCH);
|
||||
}
|
||||
|
||||
|
@ -554,7 +671,7 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
|
||||
static boolean shouldPreFilterSearchShards(ClusterState clusterState,
|
||||
SearchRequest searchRequest,
|
||||
Index[] indices,
|
||||
String[] indices,
|
||||
int numShards) {
|
||||
SearchSourceBuilder source = searchRequest.source();
|
||||
Integer preFilterShardSize = searchRequest.getPreFilterShardSize();
|
||||
|
@ -569,9 +686,9 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
&& preFilterShardSize < numShards;
|
||||
}
|
||||
|
||||
private static boolean hasReadOnlyIndices(Index[] indices, ClusterState clusterState) {
|
||||
for (Index index : indices) {
|
||||
ClusterBlockException writeBlock = clusterState.blocks().indexBlockedException(ClusterBlockLevel.WRITE, index.getName());
|
||||
private static boolean hasReadOnlyIndices(String[] indices, ClusterState clusterState) {
|
||||
for (String index : indices) {
|
||||
ClusterBlockException writeBlock = clusterState.blocks().indexBlockedException(ClusterBlockLevel.WRITE, index);
|
||||
if (writeBlock != null) {
|
||||
return true;
|
||||
}
|
||||
|
@ -579,29 +696,37 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
return false;
|
||||
}
|
||||
|
||||
static GroupShardsIterator<SearchShardIterator> mergeShardsIterators(GroupShardsIterator<ShardIterator> localShardsIterator,
|
||||
OriginalIndices localIndices,
|
||||
@Nullable String localClusterAlias,
|
||||
List<SearchShardIterator> remoteShardIterators) {
|
||||
static GroupShardsIterator<SearchShardIterator> mergeShardsIterators(List<SearchShardIterator> localShardIterators,
|
||||
List<SearchShardIterator> remoteShardIterators) {
|
||||
List<SearchShardIterator> shards = new ArrayList<>(remoteShardIterators);
|
||||
for (ShardIterator shardIterator : localShardsIterator) {
|
||||
shards.add(new SearchShardIterator(localClusterAlias, shardIterator.shardId(), shardIterator.getShardRoutings(), localIndices));
|
||||
}
|
||||
shards.addAll(localShardIterators);
|
||||
return GroupShardsIterator.sortAndCreate(shards);
|
||||
}
|
||||
|
||||
private AbstractSearchAsyncAction<? extends SearchPhaseResult> searchAsyncAction(SearchTask task, SearchRequest searchRequest,
|
||||
Executor executor,
|
||||
GroupShardsIterator<SearchShardIterator> shardIterators,
|
||||
SearchTimeProvider timeProvider,
|
||||
BiFunction<String, String, Transport.Connection> connectionLookup,
|
||||
ClusterState clusterState,
|
||||
Map<String, AliasFilter> aliasFilter,
|
||||
Map<String, Float> concreteIndexBoosts,
|
||||
Map<String, Set<String>> indexRoutings,
|
||||
ActionListener<SearchResponse> listener,
|
||||
boolean preFilter,
|
||||
SearchResponse.Clusters clusters) {
|
||||
interface SearchAsyncActionProvider {
|
||||
AbstractSearchAsyncAction<? extends SearchPhaseResult> asyncSearchAction(
|
||||
SearchTask task, SearchRequest searchRequest, Executor executor, GroupShardsIterator<SearchShardIterator> shardIterators,
|
||||
SearchTimeProvider timeProvider, BiFunction<String, String, Transport.Connection> connectionLookup,
|
||||
ClusterState clusterState, Map<String, AliasFilter> aliasFilter, Map<String, Float> concreteIndexBoosts,
|
||||
Map<String, Set<String>> indexRoutings, ActionListener<SearchResponse> listener, boolean preFilter,
|
||||
ThreadPool threadPool, SearchResponse.Clusters clusters);
|
||||
}
|
||||
|
||||
private AbstractSearchAsyncAction<? extends SearchPhaseResult> searchAsyncAction(
|
||||
SearchTask task,
|
||||
SearchRequest searchRequest,
|
||||
Executor executor,
|
||||
GroupShardsIterator<SearchShardIterator> shardIterators,
|
||||
SearchTimeProvider timeProvider,
|
||||
BiFunction<String, String, Transport.Connection> connectionLookup,
|
||||
ClusterState clusterState,
|
||||
Map<String, AliasFilter> aliasFilter,
|
||||
Map<String, Float> concreteIndexBoosts,
|
||||
Map<String, Set<String>> indexRoutings,
|
||||
ActionListener<SearchResponse> listener,
|
||||
boolean preFilter,
|
||||
ThreadPool threadPool,
|
||||
SearchResponse.Clusters clusters) {
|
||||
if (preFilter) {
|
||||
return new CanMatchPreFilterSearchPhase(logger, searchTransportService, connectionLookup,
|
||||
aliasFilter, concreteIndexBoosts, indexRoutings, executor, searchRequest, listener, shardIterators,
|
||||
|
@ -619,6 +744,7 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
indexRoutings,
|
||||
listener,
|
||||
false,
|
||||
threadPool,
|
||||
clusters);
|
||||
return new SearchPhase(action.getName()) {
|
||||
@Override
|
||||
|
@ -736,4 +862,38 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
private static RemoteTransportException wrapRemoteClusterFailure(String clusterAlias, Exception e) {
|
||||
return new RemoteTransportException("error while communicating with remote cluster [" + clusterAlias + "]", e);
|
||||
}
|
||||
|
||||
static Map<String, OriginalIndices> getIndicesFromSearchContexts(SearchContextId searchContext,
|
||||
IndicesOptions indicesOptions) {
|
||||
final Map<String, Set<String>> indices = new HashMap<>();
|
||||
for (Map.Entry<ShardId, SearchContextIdForNode> entry : searchContext.shards().entrySet()) {
|
||||
String clusterAlias = entry.getValue().getClusterAlias() == null ?
|
||||
RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY : entry.getValue().getClusterAlias();
|
||||
indices.computeIfAbsent(clusterAlias, k -> new HashSet<>()).add(entry.getKey().getIndexName());
|
||||
}
|
||||
return indices.entrySet().stream()
|
||||
.collect(Collectors.toMap(Map.Entry::getKey, e -> new OriginalIndices(e.getValue().toArray(new String[0]), indicesOptions)));
|
||||
}
|
||||
|
||||
static List<SearchShardIterator> getSearchShardsFromSearchContexts(ClusterState clusterState, OriginalIndices originalIndices,
|
||||
String localClusterAlias,
|
||||
SearchContextId searchContext,
|
||||
TimeValue keepAlive) {
|
||||
final List<SearchShardIterator> iterators = new ArrayList<>(searchContext.shards().size());
|
||||
for (Map.Entry<ShardId, SearchContextIdForNode> entry : searchContext.shards().entrySet()) {
|
||||
final ShardId shardId = entry.getKey();
|
||||
final ShardIterator shards = OperationRouting.getShards(clusterState, shardId);
|
||||
final List<ShardRouting> matchingNodeFirstRoutings = new ArrayList<>();
|
||||
for (ShardRouting shard : shards) {
|
||||
if (shard.currentNodeId().equals(entry.getValue().getNode())) {
|
||||
matchingNodeFirstRoutings.add(0, shard);
|
||||
} else {
|
||||
matchingNodeFirstRoutings.add(shard);
|
||||
}
|
||||
}
|
||||
iterators.add(new SearchShardIterator(localClusterAlias, shardId, matchingNodeFirstRoutings, originalIndices,
|
||||
entry.getValue().getSearchContextId(), keepAlive));
|
||||
}
|
||||
return iterators;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,26 +21,28 @@ package org.elasticsearch.action.search;
|
|||
|
||||
import org.apache.lucene.store.ByteArrayDataInput;
|
||||
import org.apache.lucene.store.RAMOutputStream;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.common.util.concurrent.AtomicArray;
|
||||
import org.elasticsearch.search.SearchPhaseResult;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.internal.InternalScrollSearchRequest;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
import org.elasticsearch.transport.RemoteClusterAware;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.UncheckedIOException;
|
||||
import java.util.Base64;
|
||||
|
||||
final class TransportSearchHelper {
|
||||
|
||||
private static final String INCLUDE_CONTEXT_UUID = "include_context_uuid";
|
||||
|
||||
static InternalScrollSearchRequest internalScrollSearchRequest(SearchContextId id, SearchScrollRequest request) {
|
||||
static InternalScrollSearchRequest internalScrollSearchRequest(ShardSearchContextId id, SearchScrollRequest request) {
|
||||
return new InternalScrollSearchRequest(request, id);
|
||||
}
|
||||
|
||||
static String buildScrollId(AtomicArray<? extends SearchPhaseResult> searchPhaseResults,
|
||||
boolean includeContextUUID) throws IOException {
|
||||
static String buildScrollId(AtomicArray<? extends SearchPhaseResult> searchPhaseResults, Version version) {
|
||||
boolean includeContextUUID = version.onOrAfter(Version.V_7_7_0);
|
||||
try (RAMOutputStream out = new RAMOutputStream()) {
|
||||
if (includeContextUUID) {
|
||||
out.writeString(INCLUDE_CONTEXT_UUID);
|
||||
|
@ -63,6 +65,8 @@ final class TransportSearchHelper {
|
|||
byte[] bytes = new byte[(int) out.getFilePointer()];
|
||||
out.writeTo(bytes, 0);
|
||||
return Base64.getUrlEncoder().encodeToString(bytes);
|
||||
} catch (IOException e) {
|
||||
throw new UncheckedIOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -80,7 +84,7 @@ final class TransportSearchHelper {
|
|||
includeContextUUID = false;
|
||||
type = firstChunk;
|
||||
}
|
||||
ScrollIdForNode[] context = new ScrollIdForNode[in.readVInt()];
|
||||
SearchContextIdForNode[] context = new SearchContextIdForNode[in.readVInt()];
|
||||
for (int i = 0; i < context.length; ++i) {
|
||||
final String contextUUID = includeContextUUID ? in.readString() : "";
|
||||
long id = in.readLong();
|
||||
|
@ -93,7 +97,7 @@ final class TransportSearchHelper {
|
|||
clusterAlias = target.substring(0, index);
|
||||
target = target.substring(index+1);
|
||||
}
|
||||
context[i] = new ScrollIdForNode(clusterAlias, target, new SearchContextId(contextUUID, id));
|
||||
context[i] = new SearchContextIdForNode(clusterAlias, target, new ShardSearchContextId(contextUUID, id));
|
||||
}
|
||||
if (in.getPosition() != bytes.length) {
|
||||
throw new IllegalArgumentException("Not all bytes were read");
|
||||
|
|
|
@ -27,6 +27,7 @@ import org.elasticsearch.action.support.TransportAction;
|
|||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.client.support.AbstractClient;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.tasks.TaskListener;
|
||||
|
@ -48,16 +49,18 @@ public class NodeClient extends AbstractClient {
|
|||
*/
|
||||
private Supplier<String> localNodeId;
|
||||
private RemoteClusterService remoteClusterService;
|
||||
private NamedWriteableRegistry namedWriteableRegistry;
|
||||
|
||||
public NodeClient(Settings settings, ThreadPool threadPool) {
|
||||
super(settings, threadPool);
|
||||
}
|
||||
|
||||
public void initialize(Map<ActionType, TransportAction> actions, Supplier<String> localNodeId,
|
||||
RemoteClusterService remoteClusterService) {
|
||||
RemoteClusterService remoteClusterService, NamedWriteableRegistry namedWriteableRegistry) {
|
||||
this.actions = actions;
|
||||
this.localNodeId = localNodeId;
|
||||
this.remoteClusterService = remoteClusterService;
|
||||
this.namedWriteableRegistry = namedWriteableRegistry;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -122,4 +125,9 @@ public class NodeClient extends AbstractClient {
|
|||
public Client getRemoteClusterClient(String clusterAlias) {
|
||||
return remoteClusterService.getRemoteClusterClient(threadPool(), clusterAlias);
|
||||
}
|
||||
|
||||
|
||||
public NamedWriteableRegistry getNamedWriteableRegistry() {
|
||||
return namedWriteableRegistry;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -140,6 +140,11 @@ public class OperationRouting {
|
|||
return GroupShardsIterator.sortAndCreate(new ArrayList<>(set));
|
||||
}
|
||||
|
||||
public static ShardIterator getShards(ClusterState clusterState, ShardId shardId) {
|
||||
final IndexShardRoutingTable shard = clusterState.routingTable().shardRoutingTable(shardId);
|
||||
return shard.activeInitializingShardsRandomIt();
|
||||
}
|
||||
|
||||
private static final Map<String, Set<String>> EMPTY_ROUTING = Collections.emptyMap();
|
||||
|
||||
private Set<IndexShardRoutingTable> computeTargetedShards(ClusterState clusterState, String[] concreteIndices,
|
||||
|
|
|
@ -100,6 +100,7 @@ import java.util.concurrent.locks.Lock;
|
|||
import java.util.concurrent.locks.ReentrantLock;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
import java.util.function.BiFunction;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_PRIMARY_TERM;
|
||||
|
@ -112,6 +113,7 @@ public abstract class Engine implements Closeable {
|
|||
public static final String FORCE_MERGE_UUID_KEY = "force_merge_uuid";
|
||||
public static final String MIN_RETAINED_SEQNO = "min_retained_seq_no";
|
||||
public static final String MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID = "max_unsafe_auto_id_timestamp";
|
||||
public static final String CAN_MATCH_SEARCH_SOURCE = "can_match"; // TODO: Make source of search enum?
|
||||
|
||||
protected final ShardId shardId;
|
||||
protected final String allocationId;
|
||||
|
@ -609,31 +611,17 @@ public abstract class Engine implements Closeable {
|
|||
|
||||
public abstract GetResult get(Get get, BiFunction<String, SearcherScope, Searcher> searcherFactory) throws EngineException;
|
||||
|
||||
|
||||
/**
|
||||
* Returns a new searcher instance. The consumer of this
|
||||
* API is responsible for releasing the returned searcher in a
|
||||
* safe manner, preferably in a try/finally block.
|
||||
*
|
||||
* @param source the source API or routing that triggers this searcher acquire
|
||||
*
|
||||
* @see Searcher#close()
|
||||
* Acquires a point-in-time reader that can be used to create {@link Engine.Searcher}s on demand.
|
||||
*/
|
||||
public final Searcher acquireSearcher(String source) throws EngineException {
|
||||
return acquireSearcher(source, SearcherScope.EXTERNAL);
|
||||
public final SearcherSupplier acquireSearcherSupplier(Function<Searcher, Searcher> wrapper) throws EngineException {
|
||||
return acquireSearcherSupplier(wrapper, SearcherScope.EXTERNAL);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a new searcher instance. The consumer of this
|
||||
* API is responsible for releasing the returned searcher in a
|
||||
* safe manner, preferably in a try/finally block.
|
||||
*
|
||||
* @param source the source API or routing that triggers this searcher acquire
|
||||
* @param scope the scope of this searcher ie. if the searcher will be used for get or search purposes
|
||||
*
|
||||
* @see Searcher#close()
|
||||
* Acquires a point-in-time reader that can be used to create {@link Engine.Searcher}s on demand.
|
||||
*/
|
||||
public Searcher acquireSearcher(String source, SearcherScope scope) throws EngineException {
|
||||
public SearcherSupplier acquireSearcherSupplier(Function<Searcher, Searcher> wrapper, SearcherScope scope) throws EngineException {
|
||||
/* Acquire order here is store -> manager since we need
|
||||
* to make sure that the store is not closed before
|
||||
* the searcher is acquired. */
|
||||
|
@ -642,35 +630,60 @@ public abstract class Engine implements Closeable {
|
|||
}
|
||||
Releasable releasable = store::decRef;
|
||||
try {
|
||||
assert assertSearcherIsWarmedUp(source, scope);
|
||||
ReferenceManager<ElasticsearchDirectoryReader> referenceManager = getReferenceManager(scope);
|
||||
final ElasticsearchDirectoryReader acquire = referenceManager.acquire();
|
||||
AtomicBoolean released = new AtomicBoolean(false);
|
||||
Searcher engineSearcher = new Searcher(source, acquire,
|
||||
engineConfig.getSimilarity(), engineConfig.getQueryCache(), engineConfig.getQueryCachingPolicy(),
|
||||
() -> {
|
||||
if (released.compareAndSet(false, true)) {
|
||||
ElasticsearchDirectoryReader acquire = referenceManager.acquire();
|
||||
SearcherSupplier reader = new SearcherSupplier(wrapper) {
|
||||
@Override
|
||||
public Searcher acquireSearcherInternal(String source) {
|
||||
assert assertSearcherIsWarmedUp(source, scope);
|
||||
return new Searcher(source, acquire, engineConfig.getSimilarity(), engineConfig.getQueryCache(),
|
||||
engineConfig.getQueryCachingPolicy(), () -> {});
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doClose() {
|
||||
try {
|
||||
referenceManager.release(acquire);
|
||||
} catch (IOException e) {
|
||||
throw new UncheckedIOException("failed to close", e);
|
||||
} catch (AlreadyClosedException e) {
|
||||
// This means there's a bug somewhere: don't suppress it
|
||||
throw new AssertionError(e);
|
||||
} finally {
|
||||
store.decRef();
|
||||
}
|
||||
} else {
|
||||
/* In general, readers should never be released twice or this would break reference counting. There is one rare case
|
||||
* when it might happen though: when the request and the Reaper thread would both try to release it in a very short
|
||||
* amount of time, this is why we only log a warning instead of throwing an exception. */
|
||||
logger.warn("Searcher was released twice", new IllegalStateException("Double release"));
|
||||
}
|
||||
});
|
||||
};
|
||||
releasable = null; // success - hand over the reference to the engine reader
|
||||
return engineSearcher;
|
||||
return reader;
|
||||
} catch (AlreadyClosedException ex) {
|
||||
throw ex;
|
||||
} catch (Exception ex) {
|
||||
maybeFailEngine("acquire_searcher", ex);
|
||||
maybeFailEngine("acquire_reader", ex);
|
||||
ensureOpen(ex); // throw EngineCloseException here if we are already closed
|
||||
logger.error(() -> new ParameterizedMessage("failed to acquire searcher, source {}", source), ex);
|
||||
throw new EngineException(shardId, "failed to acquire searcher, source " + source, ex);
|
||||
logger.error(() -> new ParameterizedMessage("failed to acquire reader"), ex);
|
||||
throw new EngineException(shardId, "failed to acquire reader", ex);
|
||||
} finally {
|
||||
Releasables.close(releasable);
|
||||
}
|
||||
}
|
||||
|
||||
public final Searcher acquireSearcher(String source) throws EngineException {
|
||||
return acquireSearcher(source, SearcherScope.EXTERNAL);
|
||||
}
|
||||
|
||||
public Searcher acquireSearcher(String source, SearcherScope scope) throws EngineException {
|
||||
return acquireSearcher(source, scope, Function.identity());
|
||||
}
|
||||
|
||||
public Searcher acquireSearcher(String source, SearcherScope scope, Function<Searcher, Searcher> wrapper) throws EngineException {
|
||||
SearcherSupplier releasable = null;
|
||||
try {
|
||||
SearcherSupplier reader = releasable = acquireSearcherSupplier(wrapper, scope);
|
||||
Searcher searcher = reader.acquireSearcher(source);
|
||||
releasable = null;
|
||||
return new Searcher(source, searcher.getDirectoryReader(), searcher.getSimilarity(),
|
||||
searcher.getQueryCache(), searcher.getQueryCachingPolicy(), () -> Releasables.close(searcher, reader));
|
||||
} finally {
|
||||
Releasables.close(releasable);
|
||||
}
|
||||
|
@ -1196,6 +1209,36 @@ public abstract class Engine implements Closeable {
|
|||
}
|
||||
}
|
||||
|
||||
public abstract static class SearcherSupplier implements Releasable {
|
||||
private final Function<Searcher, Searcher> wrapper;
|
||||
private final AtomicBoolean released = new AtomicBoolean(false);
|
||||
|
||||
public SearcherSupplier(Function<Searcher, Searcher> wrapper) {
|
||||
this.wrapper = wrapper;
|
||||
}
|
||||
|
||||
public final Searcher acquireSearcher(String source) {
|
||||
if (released.get()) {
|
||||
throw new AlreadyClosedException("SearcherSupplier was closed");
|
||||
}
|
||||
final Searcher searcher = acquireSearcherInternal(source);
|
||||
return CAN_MATCH_SEARCH_SOURCE.equals(source) ? searcher : wrapper.apply(searcher);
|
||||
}
|
||||
|
||||
@Override
|
||||
public final void close() {
|
||||
if (released.compareAndSet(false, true)) {
|
||||
doClose();
|
||||
} else {
|
||||
assert false : "SearchSupplier was released twice";
|
||||
}
|
||||
}
|
||||
|
||||
protected abstract void doClose();
|
||||
|
||||
protected abstract Searcher acquireSearcherInternal(String source);
|
||||
}
|
||||
|
||||
public static final class Searcher extends IndexSearcher implements Releasable {
|
||||
private final String source;
|
||||
private final Closeable onClose;
|
||||
|
|
|
@ -426,12 +426,7 @@ public class NestedQueryBuilder extends AbstractQueryBuilder<NestedQueryBuilder>
|
|||
topDocsCollector = TopScoreDocCollector.create(topN, Integer.MAX_VALUE);
|
||||
maxScoreCollector = new MaxScoreCollector();
|
||||
}
|
||||
try {
|
||||
intersect(weight, innerHitQueryWeight, MultiCollector.wrap(topDocsCollector, maxScoreCollector), ctx);
|
||||
} finally {
|
||||
clearReleasables(Lifetime.COLLECTION);
|
||||
}
|
||||
|
||||
intersect(weight, innerHitQueryWeight, MultiCollector.wrap(topDocsCollector, maxScoreCollector), ctx);
|
||||
TopDocs td = topDocsCollector.topDocs(from(), size());
|
||||
float maxScore = Float.NaN;
|
||||
if (maxScoreCollector != null) {
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.elasticsearch.common.metrics.MeanMetric;
|
|||
import org.elasticsearch.common.regex.Regex;
|
||||
import org.elasticsearch.common.util.CollectionUtils;
|
||||
import org.elasticsearch.index.shard.SearchOperationListener;
|
||||
import org.elasticsearch.search.internal.ReaderContext;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
|
||||
import java.util.HashMap;
|
||||
|
@ -147,25 +148,25 @@ public final class ShardSearchStats implements SearchOperationListener {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void onNewContext(SearchContext context) {
|
||||
public void onNewReaderContext(ReaderContext readerContext) {
|
||||
openContexts.inc();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFreeContext(SearchContext context) {
|
||||
public void onFreeReaderContext(ReaderContext readerContext) {
|
||||
openContexts.dec();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onNewScrollContext(SearchContext context) {
|
||||
public void onNewScrollContext(ReaderContext readerContext) {
|
||||
totalStats.scrollCurrent.inc();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFreeScrollContext(SearchContext context) {
|
||||
public void onFreeScrollContext(ReaderContext readerContext) {
|
||||
totalStats.scrollCurrent.dec();
|
||||
assert totalStats.scrollCurrent.count() >= 0;
|
||||
totalStats.scrollMetric.inc(TimeUnit.NANOSECONDS.toMicros(System.nanoTime() - context.getOriginNanoTime()));
|
||||
totalStats.scrollMetric.inc(TimeUnit.NANOSECONDS.toMicros(System.nanoTime() - readerContext.getStartTimeInNano()));
|
||||
}
|
||||
|
||||
static final class StatsHolder {
|
||||
|
|
|
@ -1241,12 +1241,20 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
|||
}
|
||||
|
||||
/**
|
||||
* Acquire a lightweight searcher which can be used to rewrite shard search requests.
|
||||
* Acquires a point-in-time reader that can be used to create {@link Engine.Searcher}s on demand.
|
||||
*/
|
||||
public Engine.Searcher acquireCanMatchSearcher() {
|
||||
public Engine.SearcherSupplier acquireSearcherSupplier() {
|
||||
return acquireSearcherSupplier(Engine.SearcherScope.EXTERNAL);
|
||||
}
|
||||
|
||||
/**
|
||||
* Acquires a point-in-time reader that can be used to create {@link Engine.Searcher}s on demand.
|
||||
*/
|
||||
public Engine.SearcherSupplier acquireSearcherSupplier(Engine.SearcherScope scope) {
|
||||
readAllowed();
|
||||
markSearcherAccessed();
|
||||
return getEngine().acquireSearcher("can_match", Engine.SearcherScope.EXTERNAL);
|
||||
final Engine engine = getEngine();
|
||||
return engine.acquireSearcherSupplier(this::wrapSearcher, scope);
|
||||
}
|
||||
|
||||
public Engine.Searcher acquireSearcher(String source) {
|
||||
|
@ -1261,8 +1269,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
|||
readAllowed();
|
||||
markSearcherAccessed();
|
||||
final Engine engine = getEngine();
|
||||
final Engine.Searcher searcher = engine.acquireSearcher(source, scope);
|
||||
return wrapSearcher(searcher);
|
||||
return engine.acquireSearcher(source, scope, this::wrapSearcher);
|
||||
}
|
||||
|
||||
private Engine.Searcher wrapSearcher(Engine.Searcher searcher) {
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.elasticsearch.index.shard;
|
|||
import org.apache.logging.log4j.Logger;
|
||||
import org.apache.logging.log4j.message.ParameterizedMessage;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.search.internal.ReaderContext;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
import org.elasticsearch.transport.TransportRequest;
|
||||
|
||||
|
@ -76,43 +77,43 @@ public interface SearchOperationListener {
|
|||
default void onFetchPhase(SearchContext searchContext, long tookInNanos) {}
|
||||
|
||||
/**
|
||||
* Executed when a new search context was created
|
||||
* @param context the created context
|
||||
* Executed when a new reader context was created
|
||||
* @param readerContext the created context
|
||||
*/
|
||||
default void onNewContext(SearchContext context) {}
|
||||
default void onNewReaderContext(ReaderContext readerContext) {}
|
||||
|
||||
/**
|
||||
* Executed when a previously created search context is freed.
|
||||
* Executed when a previously created reader context is freed.
|
||||
* This happens either when the search execution finishes, if the
|
||||
* execution failed or if the search context as idle for and needs to be
|
||||
* cleaned up.
|
||||
* @param context the freed search context
|
||||
* @param readerContext the freed reader context
|
||||
*/
|
||||
default void onFreeContext(SearchContext context) {}
|
||||
default void onFreeReaderContext(ReaderContext readerContext) {}
|
||||
|
||||
/**
|
||||
* Executed when a new scroll search {@link SearchContext} was created
|
||||
* @param context the created search context
|
||||
* Executed when a new scroll search {@link ReaderContext} was created
|
||||
* @param readerContext the created reader context
|
||||
*/
|
||||
default void onNewScrollContext(SearchContext context) {}
|
||||
default void onNewScrollContext(ReaderContext readerContext) {}
|
||||
|
||||
/**
|
||||
* Executed when a scroll search {@link SearchContext} is freed.
|
||||
* This happens either when the scroll search execution finishes, if the
|
||||
* execution failed or if the search context as idle for and needs to be
|
||||
* cleaned up.
|
||||
* @param context the freed search context
|
||||
* @param readerContext the freed search context
|
||||
*/
|
||||
default void onFreeScrollContext(SearchContext context) {}
|
||||
default void onFreeScrollContext(ReaderContext readerContext) {}
|
||||
|
||||
/**
|
||||
* Executed prior to using a {@link SearchContext} that has been retrieved
|
||||
* Executed prior to using a {@link ReaderContext} that has been retrieved
|
||||
* from the active contexts. If the context is deemed invalid a runtime
|
||||
* exception can be thrown, which will prevent the context from being used.
|
||||
* @param context the context retrieved from the active contexts
|
||||
* @param readerContext The reader context used by this request.
|
||||
* @param transportRequest the request that is going to use the search context
|
||||
*/
|
||||
default void validateSearchContext(SearchContext context, TransportRequest transportRequest) {}
|
||||
default void validateSearchContext(ReaderContext readerContext, TransportRequest transportRequest) {}
|
||||
|
||||
/**
|
||||
* A Composite listener that multiplexes calls to each of the listeners methods.
|
||||
|
@ -193,10 +194,10 @@ public interface SearchOperationListener {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void onNewContext(SearchContext context) {
|
||||
public void onNewReaderContext(ReaderContext readerContext) {
|
||||
for (SearchOperationListener listener : listeners) {
|
||||
try {
|
||||
listener.onNewContext(context);
|
||||
listener.onNewReaderContext(readerContext);
|
||||
} catch (Exception e) {
|
||||
logger.warn(() -> new ParameterizedMessage("onNewContext listener [{}] failed", listener), e);
|
||||
}
|
||||
|
@ -204,10 +205,10 @@ public interface SearchOperationListener {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void onFreeContext(SearchContext context) {
|
||||
public void onFreeReaderContext(ReaderContext readerContext) {
|
||||
for (SearchOperationListener listener : listeners) {
|
||||
try {
|
||||
listener.onFreeContext(context);
|
||||
listener.onFreeReaderContext(readerContext);
|
||||
} catch (Exception e) {
|
||||
logger.warn(() -> new ParameterizedMessage("onFreeContext listener [{}] failed", listener), e);
|
||||
}
|
||||
|
@ -215,10 +216,10 @@ public interface SearchOperationListener {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void onNewScrollContext(SearchContext context) {
|
||||
public void onNewScrollContext(ReaderContext readerContext) {
|
||||
for (SearchOperationListener listener : listeners) {
|
||||
try {
|
||||
listener.onNewScrollContext(context);
|
||||
listener.onNewScrollContext(readerContext);
|
||||
} catch (Exception e) {
|
||||
logger.warn(() -> new ParameterizedMessage("onNewScrollContext listener [{}] failed", listener), e);
|
||||
}
|
||||
|
@ -226,10 +227,10 @@ public interface SearchOperationListener {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void onFreeScrollContext(SearchContext context) {
|
||||
public void onFreeScrollContext(ReaderContext readerContext) {
|
||||
for (SearchOperationListener listener : listeners) {
|
||||
try {
|
||||
listener.onFreeScrollContext(context);
|
||||
listener.onFreeScrollContext(readerContext);
|
||||
} catch (Exception e) {
|
||||
logger.warn(() -> new ParameterizedMessage("onFreeScrollContext listener [{}] failed", listener), e);
|
||||
}
|
||||
|
@ -237,11 +238,11 @@ public interface SearchOperationListener {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void validateSearchContext(SearchContext context, TransportRequest request) {
|
||||
public void validateSearchContext(ReaderContext readerContext, TransportRequest request) {
|
||||
Exception exception = null;
|
||||
for (SearchOperationListener listener : listeners) {
|
||||
try {
|
||||
listener.validateSearchContext(context, request);
|
||||
listener.validateSearchContext(readerContext, request);
|
||||
} catch (Exception e) {
|
||||
exception = ExceptionsHelper.useOrSuppress(exception, e);
|
||||
}
|
||||
|
|
|
@ -694,8 +694,8 @@ public class Node implements Closeable {
|
|||
resourcesToClose.add(injector.getInstance(PeerRecoverySourceService.class));
|
||||
this.pluginLifecycleComponents = Collections.unmodifiableList(pluginLifecycleComponents);
|
||||
client.initialize(injector.getInstance(new Key<Map<ActionType, TransportAction>>() {}),
|
||||
() -> clusterService.localNode().getId(), transportService.getRemoteClusterService());
|
||||
|
||||
() -> clusterService.localNode().getId(), transportService.getRemoteClusterService(),
|
||||
namedWriteableRegistry);
|
||||
logger.debug("initializing HTTP handlers ...");
|
||||
actionModule.initRestHandlers(() -> clusterService.state().nodes());
|
||||
logger.info("initialized");
|
||||
|
|
|
@ -20,11 +20,13 @@
|
|||
package org.elasticsearch.rest.action.search;
|
||||
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchContextId;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.support.IndicesOptions;
|
||||
import org.elasticsearch.client.node.NodeClient;
|
||||
import org.elasticsearch.common.Booleans;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
||||
import org.elasticsearch.common.logging.DeprecationLogger;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.index.query.QueryBuilder;
|
||||
|
@ -108,8 +110,12 @@ public class RestSearchAction extends BaseRestHandler {
|
|||
* company.
|
||||
*/
|
||||
IntConsumer setSize = size -> searchRequest.source().size(size);
|
||||
request.withContentOrSourceParamParserOrNull(parser ->
|
||||
parseSearchRequest(searchRequest, request, parser, setSize));
|
||||
request.withContentOrSourceParamParserOrNull(parser -> {
|
||||
parseSearchRequest(searchRequest, request, parser, setSize);
|
||||
if (searchRequest.pointInTimeBuilder() != null) {
|
||||
preparePointInTime(searchRequest, client.getNamedWriteableRegistry());
|
||||
}
|
||||
});
|
||||
|
||||
return channel -> {
|
||||
RestCancellableNodeClient cancelClient = new RestCancellableNodeClient(client, request.getHttpChannel());
|
||||
|
@ -298,6 +304,17 @@ public class RestSearchAction extends BaseRestHandler {
|
|||
}
|
||||
}
|
||||
|
||||
static void preparePointInTime(SearchRequest request, NamedWriteableRegistry namedWriteableRegistry) {
|
||||
assert request.pointInTimeBuilder() != null;
|
||||
final IndicesOptions indicesOptions = request.indicesOptions();
|
||||
final IndicesOptions stricterIndicesOptions = IndicesOptions.fromOptions(
|
||||
indicesOptions.ignoreUnavailable(), indicesOptions.allowNoIndices(), false, false, false,
|
||||
true, true, indicesOptions.ignoreThrottled());
|
||||
request.indicesOptions(stricterIndicesOptions);
|
||||
final SearchContextId searchContextId = SearchContextId.decode(namedWriteableRegistry, request.pointInTimeBuilder().getId());
|
||||
request.indices(searchContextId.getActualIndices());
|
||||
}
|
||||
|
||||
/**
|
||||
* Modify the search request to accurately count the total hits that match the query
|
||||
* if {@link #TOTAL_HITS_AS_INT_PARAM} is set.
|
||||
|
|
|
@ -63,9 +63,10 @@ import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
|
|||
import org.elasticsearch.search.fetch.subphase.ScriptFieldsContext;
|
||||
import org.elasticsearch.search.fetch.subphase.highlight.SearchHighlightContext;
|
||||
import org.elasticsearch.search.internal.ContextIndexSearcher;
|
||||
import org.elasticsearch.search.internal.ReaderContext;
|
||||
import org.elasticsearch.search.internal.ScrollContext;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchRequest;
|
||||
import org.elasticsearch.search.profile.Profilers;
|
||||
import org.elasticsearch.search.query.QueryPhaseExecutionException;
|
||||
|
@ -87,12 +88,12 @@ import java.util.function.LongSupplier;
|
|||
|
||||
final class DefaultSearchContext extends SearchContext {
|
||||
|
||||
private final SearchContextId id;
|
||||
private final ReaderContext readerContext;
|
||||
private final Engine.Searcher engineSearcher;
|
||||
private final ShardSearchRequest request;
|
||||
private final SearchShardTarget shardTarget;
|
||||
private final LongSupplier relativeTimeSupplier;
|
||||
private SearchType searchType;
|
||||
private final Engine.Searcher engineSearcher;
|
||||
private final BigArrays bigArrays;
|
||||
private final IndexShard indexShard;
|
||||
private final ClusterService clusterService;
|
||||
|
@ -107,7 +108,6 @@ final class DefaultSearchContext extends SearchContext {
|
|||
// terminate after count
|
||||
private int terminateAfter = DEFAULT_TERMINATE_AFTER;
|
||||
private List<String> groupStats;
|
||||
private ScrollContext scrollContext;
|
||||
private boolean explain;
|
||||
private boolean version = false; // by default, we don't return versions
|
||||
private boolean seqAndPrimaryTerm = false;
|
||||
|
@ -149,9 +149,6 @@ final class DefaultSearchContext extends SearchContext {
|
|||
private SearchHighlightContext highlight;
|
||||
private SuggestionSearchContext suggest;
|
||||
private List<RescoreContext> rescore;
|
||||
private volatile long keepAlive;
|
||||
private final long originNanoTime = System.nanoTime();
|
||||
private volatile long lastAccessTime = -1;
|
||||
private Profilers profilers;
|
||||
|
||||
private final Map<String, SearchExtBuilder> searchExtBuilders = new HashMap<>();
|
||||
|
@ -159,30 +156,36 @@ final class DefaultSearchContext extends SearchContext {
|
|||
private final QueryShardContext queryShardContext;
|
||||
private final FetchPhase fetchPhase;
|
||||
|
||||
DefaultSearchContext(SearchContextId id, ShardSearchRequest request, SearchShardTarget shardTarget,
|
||||
Engine.Searcher engineSearcher, ClusterService clusterService, IndexService indexService,
|
||||
IndexShard indexShard, BigArrays bigArrays, LongSupplier relativeTimeSupplier, TimeValue timeout,
|
||||
FetchPhase fetchPhase, boolean lowLevelCancellation, Version minNodeVersion) throws IOException {
|
||||
this.id = id;
|
||||
DefaultSearchContext(ReaderContext readerContext,
|
||||
ShardSearchRequest request,
|
||||
SearchShardTarget shardTarget,
|
||||
ClusterService clusterService,
|
||||
BigArrays bigArrays,
|
||||
LongSupplier relativeTimeSupplier,
|
||||
TimeValue timeout,
|
||||
FetchPhase fetchPhase,
|
||||
boolean lowLevelCancellation,
|
||||
Version minNodeVersion) throws IOException {
|
||||
this.readerContext = readerContext;
|
||||
this.request = request;
|
||||
this.fetchPhase = fetchPhase;
|
||||
this.searchType = request.searchType();
|
||||
this.shardTarget = shardTarget;
|
||||
this.engineSearcher = engineSearcher;
|
||||
// SearchContexts use a BigArrays that can circuit break
|
||||
this.bigArrays = bigArrays.withCircuitBreaking();
|
||||
this.dfsResult = new DfsSearchResult(id, shardTarget);
|
||||
this.queryResult = new QuerySearchResult(id, shardTarget);
|
||||
this.fetchResult = new FetchSearchResult(id, shardTarget);
|
||||
this.indexShard = indexShard;
|
||||
this.indexService = indexService;
|
||||
this.dfsResult = new DfsSearchResult(readerContext.id(), shardTarget, request);
|
||||
this.queryResult = new QuerySearchResult(readerContext.id(), shardTarget, request);
|
||||
this.fetchResult = new FetchSearchResult(readerContext.id(), shardTarget);
|
||||
this.indexService = readerContext.indexService();
|
||||
this.indexShard = readerContext.indexShard();
|
||||
this.clusterService = clusterService;
|
||||
this.engineSearcher = readerContext.acquireSearcher("search");
|
||||
this.searcher = new ContextIndexSearcher(engineSearcher.getIndexReader(), engineSearcher.getSimilarity(),
|
||||
engineSearcher.getQueryCache(), engineSearcher.getQueryCachingPolicy(), lowLevelCancellation);
|
||||
this.relativeTimeSupplier = relativeTimeSupplier;
|
||||
this.timeout = timeout;
|
||||
this.minNodeVersion = minNodeVersion;
|
||||
queryShardContext = indexService.newQueryShardContext(request.shardId().id(), searcher,
|
||||
queryShardContext = indexService.newQueryShardContext(request.shardId().id(), this.searcher,
|
||||
request::nowInMillis, shardTarget.getClusterAlias());
|
||||
queryShardContext.setTypes(request.types());
|
||||
queryBoost = request.indexBoost();
|
||||
|
@ -208,7 +211,7 @@ final class DefaultSearchContext extends SearchContext {
|
|||
int maxResultWindow = indexService.getIndexSettings().getMaxResultWindow();
|
||||
|
||||
if (resultWindow > maxResultWindow) {
|
||||
if (scrollContext == null) {
|
||||
if (scrollContext() == null) {
|
||||
throw new IllegalArgumentException(
|
||||
"Result window is too large, from + size must be less than or equal to: [" + maxResultWindow + "] but was ["
|
||||
+ resultWindow + "]. See the scroll api for a more efficient way to request large data sets. "
|
||||
|
@ -225,7 +228,7 @@ final class DefaultSearchContext extends SearchContext {
|
|||
throw new IllegalArgumentException("Cannot use [sort] option in conjunction with [rescore].");
|
||||
}
|
||||
int maxWindow = indexService.getIndexSettings().getMaxRescoreWindow();
|
||||
for (RescoreContext rescoreContext: rescore) {
|
||||
for (RescoreContext rescoreContext: rescore()) {
|
||||
if (rescoreContext.getWindowSize() > maxWindow) {
|
||||
throw new IllegalArgumentException("Rescore window [" + rescoreContext.getWindowSize() + "] is too large. "
|
||||
+ "It must be less than [" + maxWindow + "]. This prevents allocating massive heaps for storing the results "
|
||||
|
@ -321,13 +324,13 @@ final class DefaultSearchContext extends SearchContext {
|
|||
}
|
||||
|
||||
@Override
|
||||
public SearchContextId id() {
|
||||
return this.id;
|
||||
public ShardSearchContextId id() {
|
||||
return readerContext.id();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String source() {
|
||||
return engineSearcher.source();
|
||||
return "search";
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -355,20 +358,9 @@ final class DefaultSearchContext extends SearchContext {
|
|||
return queryBoost;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getOriginNanoTime() {
|
||||
return originNanoTime;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ScrollContext scrollContext() {
|
||||
return this.scrollContext;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SearchContext scrollContext(ScrollContext scrollContext) {
|
||||
this.scrollContext = scrollContext;
|
||||
return this;
|
||||
return readerContext.scrollContext();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -768,26 +760,6 @@ final class DefaultSearchContext extends SearchContext {
|
|||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void accessed(long accessTime) {
|
||||
this.lastAccessTime = accessTime;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long lastAccessTime() {
|
||||
return this.lastAccessTime;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long keepAlive() {
|
||||
return this.keepAlive;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void keepAlive(long keepAlive) {
|
||||
this.keepAlive = keepAlive;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DfsSearchResult dfsResult() {
|
||||
return dfsResult;
|
||||
|
@ -856,4 +828,9 @@ final class DefaultSearchContext extends SearchContext {
|
|||
public boolean isCancelled() {
|
||||
return task.isCancelled();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReaderContext readerContext() {
|
||||
return readerContext;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,57 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Since {@link org.elasticsearch.search.internal.SearchContext} no longer hold the states of search, the top K results
|
||||
* (i.e., documents that will be rescored by query rescorers) need to be serialized/ deserialized between search phases.
|
||||
* A {@link RescoreDocIds} encapsulates the top K results for each rescorer by its ordinal index.
|
||||
*/
|
||||
public final class RescoreDocIds implements Writeable {
|
||||
public static final RescoreDocIds EMPTY = new RescoreDocIds(Collections.emptyMap());
|
||||
|
||||
private final Map<Integer, Set<Integer>> docIds;
|
||||
|
||||
public RescoreDocIds(Map<Integer, Set<Integer>> docIds) {
|
||||
this.docIds = docIds;
|
||||
}
|
||||
|
||||
public RescoreDocIds(StreamInput in) throws IOException {
|
||||
docIds = in.readMap(StreamInput::readVInt, i -> i.readSet(StreamInput::readVInt));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
out.writeMap(docIds, StreamOutput::writeVInt, (o, v) -> o.writeCollection(v, StreamOutput::writeVInt));
|
||||
}
|
||||
|
||||
public Set<Integer> getId(int index) {
|
||||
return docIds.get(index);
|
||||
}
|
||||
}
|
|
@ -23,20 +23,20 @@ import org.elasticsearch.ElasticsearchException;
|
|||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class SearchContextMissingException extends ElasticsearchException {
|
||||
|
||||
private final SearchContextId contextId;
|
||||
private final ShardSearchContextId contextId;
|
||||
|
||||
public SearchContextMissingException(SearchContextId contextId) {
|
||||
public SearchContextMissingException(ShardSearchContextId contextId) {
|
||||
super("No search context found for id [" + contextId.getId() + "]");
|
||||
this.contextId = contextId;
|
||||
}
|
||||
|
||||
public SearchContextId contextId() {
|
||||
public ShardSearchContextId contextId() {
|
||||
return this.contextId;
|
||||
}
|
||||
|
||||
|
@ -47,7 +47,7 @@ public class SearchContextMissingException extends ElasticsearchException {
|
|||
|
||||
public SearchContextMissingException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
contextId = new SearchContextId(in);
|
||||
contextId = new ShardSearchContextId(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -23,7 +23,8 @@ import org.elasticsearch.common.Nullable;
|
|||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.search.fetch.FetchSearchResult;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchRequest;
|
||||
import org.elasticsearch.search.query.QuerySearchResult;
|
||||
import org.elasticsearch.transport.TransportResponse;
|
||||
|
||||
|
@ -41,7 +42,9 @@ public abstract class SearchPhaseResult extends TransportResponse {
|
|||
|
||||
private SearchShardTarget searchShardTarget;
|
||||
private int shardIndex = -1;
|
||||
protected SearchContextId contextId;
|
||||
protected ShardSearchContextId contextId;
|
||||
private ShardSearchRequest shardSearchRequest;
|
||||
private RescoreDocIds rescoreDocIds = RescoreDocIds.EMPTY;
|
||||
|
||||
protected SearchPhaseResult() {
|
||||
|
||||
|
@ -56,7 +59,7 @@ public abstract class SearchPhaseResult extends TransportResponse {
|
|||
* or <code>null</code> if no context was created.
|
||||
*/
|
||||
@Nullable
|
||||
public SearchContextId getContextId() {
|
||||
public ShardSearchContextId getContextId() {
|
||||
return contextId;
|
||||
}
|
||||
|
||||
|
@ -94,6 +97,23 @@ public abstract class SearchPhaseResult extends TransportResponse {
|
|||
*/
|
||||
public FetchSearchResult fetchResult() { return null; }
|
||||
|
||||
@Nullable
|
||||
public ShardSearchRequest getShardSearchRequest() {
|
||||
return shardSearchRequest;
|
||||
}
|
||||
|
||||
public void setShardSearchRequest(ShardSearchRequest shardSearchRequest) {
|
||||
this.shardSearchRequest = shardSearchRequest;
|
||||
}
|
||||
|
||||
public RescoreDocIds getRescoreDocIds() {
|
||||
return rescoreDocIds;
|
||||
}
|
||||
|
||||
public void setRescoreDocIds(RescoreDocIds rescoreDocIds) {
|
||||
this.rescoreDocIds = rescoreDocIds;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
// TODO: this seems wrong, SearchPhaseResult should have a writeTo?
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -112,8 +112,6 @@ public class AggregationPhase {
|
|||
context.searcher().search(query, collector);
|
||||
} catch (Exception e) {
|
||||
throw new QueryPhaseExecutionException(context.shardTarget(), "Failed to execute global aggregators", e);
|
||||
} finally {
|
||||
context.clearReleasables(SearchContext.Lifetime.COLLECTION);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -27,7 +27,6 @@ import org.elasticsearch.indices.breaker.CircuitBreakerService;
|
|||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
import org.elasticsearch.search.internal.SearchContext.Lifetime;
|
||||
import org.elasticsearch.search.query.QueryPhaseExecutionException;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -77,7 +76,7 @@ public abstract class AggregatorBase extends Aggregator {
|
|||
this.breakerService = context.bigArrays().breakerService();
|
||||
assert factories != null : "sub-factories provided to BucketAggregator must not be null, use AggragatorFactories.EMPTY instead";
|
||||
this.subAggregators = factories.createSubAggregators(context, this, subAggregatorCardinality);
|
||||
context.addReleasable(this, Lifetime.PHASE);
|
||||
context.addReleasable(this);
|
||||
final SearchShardTarget shardTarget = context.shardTarget();
|
||||
// Register a safeguard to highlight any invalid construction logic (call to this constructor without subsequent preCollection call)
|
||||
collectableSubAggregators = new BucketCollector() {
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
|
|||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.common.logging.DeprecationLogger;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.xcontent.ObjectParser;
|
||||
import org.elasticsearch.common.xcontent.ToXContentFragment;
|
||||
import org.elasticsearch.common.xcontent.ToXContentObject;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
|
@ -113,6 +114,7 @@ public final class SearchSourceBuilder implements Writeable, ToXContentObject, R
|
|||
public static final ParseField SEARCH_AFTER = new ParseField("search_after");
|
||||
public static final ParseField COLLAPSE = new ParseField("collapse");
|
||||
public static final ParseField SLICE = new ParseField("slice");
|
||||
public static final ParseField POINT_IN_TIME = new ParseField("pit");
|
||||
|
||||
public static SearchSourceBuilder fromXContent(XContentParser parser) throws IOException {
|
||||
return fromXContent(parser, true);
|
||||
|
@ -191,6 +193,8 @@ public final class SearchSourceBuilder implements Writeable, ToXContentObject, R
|
|||
|
||||
private CollapseBuilder collapse = null;
|
||||
|
||||
private PointInTimeBuilder pointInTimeBuilder = null;
|
||||
|
||||
/**
|
||||
* Constructs a new search source builder.
|
||||
*/
|
||||
|
@ -269,6 +273,9 @@ public final class SearchSourceBuilder implements Writeable, ToXContentObject, R
|
|||
fetchFields = in.readList(FieldAndFormat::new);
|
||||
}
|
||||
}
|
||||
if (in.getVersion().onOrAfter(Version.V_7_10_0)) {
|
||||
pointInTimeBuilder = in.readOptionalWriteable(PointInTimeBuilder::new);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -341,6 +348,9 @@ public final class SearchSourceBuilder implements Writeable, ToXContentObject, R
|
|||
out.writeList(fetchFields);
|
||||
}
|
||||
}
|
||||
if (out.getVersion().onOrAfter(Version.V_7_10_0)) {
|
||||
out.writeOptionalWriteable(pointInTimeBuilder);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -985,6 +995,21 @@ public final class SearchSourceBuilder implements Writeable, ToXContentObject, R
|
|||
&& queryBuilder == null && aggregations == null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the point in time that is configured with this query
|
||||
*/
|
||||
public PointInTimeBuilder pointInTimeBuilder() {
|
||||
return pointInTimeBuilder;
|
||||
}
|
||||
|
||||
/**
|
||||
* Specify a point in time that this query should execute against.
|
||||
*/
|
||||
public SearchSourceBuilder pointInTimeBuilder(PointInTimeBuilder builder) {
|
||||
this.pointInTimeBuilder = builder;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Rewrites this search source builder into its primitive form. e.g. by
|
||||
* rewriting the QueryBuilder. If the builder did not change the identity
|
||||
|
@ -1068,6 +1093,7 @@ public final class SearchSourceBuilder implements Writeable, ToXContentObject, R
|
|||
rewrittenBuilder.version = version;
|
||||
rewrittenBuilder.seqNoAndPrimaryTerm = seqNoAndPrimaryTerm;
|
||||
rewrittenBuilder.collapse = collapse;
|
||||
rewrittenBuilder.pointInTimeBuilder = pointInTimeBuilder;
|
||||
return rewrittenBuilder;
|
||||
}
|
||||
|
||||
|
@ -1189,6 +1215,8 @@ public final class SearchSourceBuilder implements Writeable, ToXContentObject, R
|
|||
sliceBuilder = SliceBuilder.fromXContent(parser);
|
||||
} else if (COLLAPSE.match(currentFieldName, parser.getDeprecationHandler())) {
|
||||
collapse = CollapseBuilder.fromXContent(parser);
|
||||
} else if (POINT_IN_TIME.match(currentFieldName, parser.getDeprecationHandler())) {
|
||||
pointInTimeBuilder = PointInTimeBuilder.fromXContent(parser);
|
||||
} else {
|
||||
throw new ParsingException(parser.getTokenLocation(), "Unknown key for a " + token + " in [" + currentFieldName + "].",
|
||||
parser.getTokenLocation());
|
||||
|
@ -1393,6 +1421,9 @@ public final class SearchSourceBuilder implements Writeable, ToXContentObject, R
|
|||
if (collapse != null) {
|
||||
builder.field(COLLAPSE.getPreferredName(), collapse);
|
||||
}
|
||||
if (pointInTimeBuilder != null) {
|
||||
builder.field(POINT_IN_TIME.getPreferredName(), pointInTimeBuilder);
|
||||
}
|
||||
return builder;
|
||||
}
|
||||
|
||||
|
@ -1605,7 +1636,7 @@ public final class SearchSourceBuilder implements Writeable, ToXContentObject, R
|
|||
return Objects.hash(aggregations, explain, fetchSourceContext, fetchFields, docValueFields, storedFieldsContext, from,
|
||||
highlightBuilder, indexBoosts, minScore, postQueryBuilder, queryBuilder, rescoreBuilders, scriptFields, size,
|
||||
sorts, searchAfterBuilder, sliceBuilder, stats, suggestBuilder, terminateAfter, timeout, trackScores, version,
|
||||
seqNoAndPrimaryTerm, profile, extBuilders, collapse, trackTotalHitsUpTo);
|
||||
seqNoAndPrimaryTerm, profile, extBuilders, collapse, trackTotalHitsUpTo, pointInTimeBuilder);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1645,7 +1676,8 @@ public final class SearchSourceBuilder implements Writeable, ToXContentObject, R
|
|||
&& Objects.equals(profile, other.profile)
|
||||
&& Objects.equals(extBuilders, other.extBuilders)
|
||||
&& Objects.equals(collapse, other.collapse)
|
||||
&& Objects.equals(trackTotalHitsUpTo, other.trackTotalHitsUpTo);
|
||||
&& Objects.equals(trackTotalHitsUpTo, other.trackTotalHitsUpTo)
|
||||
&& Objects.equals(pointInTimeBuilder, other.pointInTimeBuilder);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1660,4 +1692,81 @@ public final class SearchSourceBuilder implements Writeable, ToXContentObject, R
|
|||
throw new ElasticsearchException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Specify whether this search should use specific reader contexts instead of the latest ones.
|
||||
*/
|
||||
public static final class PointInTimeBuilder implements Writeable, ToXContentObject {
|
||||
private static final ParseField ID_FIELD = new ParseField("id");
|
||||
private static final ParseField KEEP_ALIVE_FIELD = new ParseField("keep_alive");
|
||||
private static final ObjectParser<XContentParams, Void> PARSER;
|
||||
|
||||
static {
|
||||
PARSER = new ObjectParser<>(POINT_IN_TIME.getPreferredName(), XContentParams::new);
|
||||
PARSER.declareString((params, id) -> params.id = id, ID_FIELD);
|
||||
PARSER.declareField((params, keepAlive) -> params.keepAlive = keepAlive,
|
||||
(p, c) -> TimeValue.parseTimeValue(p.text(), KEEP_ALIVE_FIELD.getPreferredName()),
|
||||
KEEP_ALIVE_FIELD, ObjectParser.ValueType.STRING);
|
||||
}
|
||||
|
||||
private static final class XContentParams {
|
||||
private String id;
|
||||
private TimeValue keepAlive;
|
||||
}
|
||||
|
||||
private final String id;
|
||||
private final TimeValue keepAlive;
|
||||
|
||||
public PointInTimeBuilder(String id, TimeValue keepAlive) {
|
||||
this.id = Objects.requireNonNull(id);
|
||||
this.keepAlive = Objects.requireNonNull(keepAlive);
|
||||
}
|
||||
|
||||
public PointInTimeBuilder(StreamInput in) throws IOException {
|
||||
id = in.readString();
|
||||
keepAlive = in.readTimeValue();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
out.writeString(id);
|
||||
out.writeTimeValue(keepAlive);
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.field(ID_FIELD.getPreferredName(), id);
|
||||
builder.field(KEEP_ALIVE_FIELD.getPreferredName(), keepAlive);
|
||||
return builder;
|
||||
}
|
||||
|
||||
public static PointInTimeBuilder fromXContent(XContentParser parser) throws IOException {
|
||||
final XContentParams params = PARSER.parse(parser, null);
|
||||
if (params.id == null || params.keepAlive == null) {
|
||||
throw new IllegalArgumentException("id and keep_alive must be specified");
|
||||
}
|
||||
return new PointInTimeBuilder(params.id, params.keepAlive);
|
||||
}
|
||||
|
||||
public TimeValue getKeepAlive() {
|
||||
return keepAlive;
|
||||
}
|
||||
|
||||
public String getId() {
|
||||
return id;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
final PointInTimeBuilder that = (PointInTimeBuilder) o;
|
||||
return Objects.equals(id, that.id) && Objects.equals(keepAlive, that.keepAlive);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(id, keepAlive);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -31,7 +31,8 @@ import org.elasticsearch.common.io.stream.StreamInput;
|
|||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.search.SearchPhaseResult;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchRequest;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
|
@ -46,7 +47,7 @@ public class DfsSearchResult extends SearchPhaseResult {
|
|||
|
||||
public DfsSearchResult(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
contextId = new SearchContextId(in);
|
||||
contextId = new ShardSearchContextId(in);
|
||||
int termsSize = in.readVInt();
|
||||
if (termsSize == 0) {
|
||||
terms = EMPTY_TERMS;
|
||||
|
@ -60,11 +61,15 @@ public class DfsSearchResult extends SearchPhaseResult {
|
|||
fieldStatistics = readFieldStats(in);
|
||||
|
||||
maxDoc = in.readVInt();
|
||||
if (in.getVersion().onOrAfter(Version.V_7_10_0)) {
|
||||
setShardSearchRequest(in.readOptionalWriteable(ShardSearchRequest::new));
|
||||
}
|
||||
}
|
||||
|
||||
public DfsSearchResult(SearchContextId contextId, SearchShardTarget shardTarget) {
|
||||
public DfsSearchResult(ShardSearchContextId contextId, SearchShardTarget shardTarget, ShardSearchRequest shardSearchRequest) {
|
||||
this.setSearchShardTarget(shardTarget);
|
||||
this.contextId = contextId;
|
||||
setShardSearchRequest(shardSearchRequest);
|
||||
}
|
||||
|
||||
public DfsSearchResult maxDoc(int maxDoc) {
|
||||
|
@ -99,7 +104,7 @@ public class DfsSearchResult extends SearchPhaseResult {
|
|||
return fieldStatistics;
|
||||
}
|
||||
|
||||
@Override
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
contextId.writeTo(out);
|
||||
out.writeVInt(terms.length);
|
||||
|
@ -110,6 +115,9 @@ public class DfsSearchResult extends SearchPhaseResult {
|
|||
writeTermStats(out, termStatistics);
|
||||
writeFieldStats(out, fieldStatistics);
|
||||
out.writeVInt(maxDoc);
|
||||
if (out.getVersion().onOrAfter(Version.V_7_10_0)) {
|
||||
out.writeOptionalWriteable(getShardSearchRequest());
|
||||
}
|
||||
}
|
||||
|
||||
public static void writeFieldStats(StreamOutput out, ObjectObjectHashMap<String,
|
||||
|
|
|
@ -25,7 +25,7 @@ import org.elasticsearch.search.SearchHit;
|
|||
import org.elasticsearch.search.SearchHits;
|
||||
import org.elasticsearch.search.SearchPhaseResult;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
import org.elasticsearch.search.query.QuerySearchResult;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -41,11 +41,11 @@ public final class FetchSearchResult extends SearchPhaseResult {
|
|||
|
||||
public FetchSearchResult(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
contextId = new SearchContextId(in);
|
||||
contextId = new ShardSearchContextId(in);
|
||||
hits = new SearchHits(in);
|
||||
}
|
||||
|
||||
public FetchSearchResult(SearchContextId id, SearchShardTarget shardTarget) {
|
||||
public FetchSearchResult(ShardSearchContextId id, SearchShardTarget shardTarget) {
|
||||
this.contextId = id;
|
||||
setSearchShardTarget(shardTarget);
|
||||
}
|
||||
|
|
|
@ -23,7 +23,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
|
|||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.search.SearchPhaseResult;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
import org.elasticsearch.search.query.QuerySearchResult;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -45,7 +45,7 @@ public final class QueryFetchSearchResult extends SearchPhaseResult {
|
|||
}
|
||||
|
||||
@Override
|
||||
public SearchContextId getContextId() {
|
||||
public ShardSearchContextId getContextId() {
|
||||
return queryResult.getContextId();
|
||||
}
|
||||
|
||||
|
|
|
@ -23,10 +23,14 @@ import com.carrotsearch.hppc.IntArrayList;
|
|||
import org.apache.lucene.search.FieldDoc;
|
||||
import org.apache.lucene.search.ScoreDoc;
|
||||
import org.elasticsearch.action.search.SearchShardTask;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.lucene.Lucene;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.RescoreDocIds;
|
||||
import org.elasticsearch.search.dfs.AggregatedDfs;
|
||||
import org.elasticsearch.search.internal.ShardSearchRequest;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.tasks.TaskId;
|
||||
import org.elasticsearch.transport.TransportRequest;
|
||||
|
@ -40,7 +44,7 @@ import java.util.Map;
|
|||
*/
|
||||
public class ShardFetchRequest extends TransportRequest {
|
||||
|
||||
private SearchContextId contextId;
|
||||
private ShardSearchContextId contextId;
|
||||
|
||||
private int[] docIds;
|
||||
|
||||
|
@ -48,10 +52,7 @@ public class ShardFetchRequest extends TransportRequest {
|
|||
|
||||
private ScoreDoc lastEmittedDoc;
|
||||
|
||||
public ShardFetchRequest() {
|
||||
}
|
||||
|
||||
public ShardFetchRequest(SearchContextId contextId, IntArrayList list, ScoreDoc lastEmittedDoc) {
|
||||
public ShardFetchRequest(ShardSearchContextId contextId, IntArrayList list, ScoreDoc lastEmittedDoc) {
|
||||
this.contextId = contextId;
|
||||
this.docIds = list.buffer;
|
||||
this.size = list.size();
|
||||
|
@ -60,7 +61,7 @@ public class ShardFetchRequest extends TransportRequest {
|
|||
|
||||
public ShardFetchRequest(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
contextId = new SearchContextId(in);
|
||||
contextId = new ShardSearchContextId(in);
|
||||
size = in.readVInt();
|
||||
docIds = new int[size];
|
||||
for (int i = 0; i < size; i++) {
|
||||
|
@ -95,7 +96,7 @@ public class ShardFetchRequest extends TransportRequest {
|
|||
}
|
||||
}
|
||||
|
||||
public SearchContextId contextId() {
|
||||
public ShardSearchContextId contextId() {
|
||||
return contextId;
|
||||
}
|
||||
|
||||
|
@ -121,4 +122,18 @@ public class ShardFetchRequest extends TransportRequest {
|
|||
return "id[" + contextId + "], size[" + size + "], lastEmittedDoc[" + lastEmittedDoc + "]";
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public ShardSearchRequest getShardSearchRequest() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public RescoreDocIds getRescoreDocIds() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public AggregatedDfs getAggregatedDfs() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,12 +21,16 @@ package org.elasticsearch.search.fetch;
|
|||
|
||||
import com.carrotsearch.hppc.IntArrayList;
|
||||
import org.apache.lucene.search.ScoreDoc;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.IndicesRequest;
|
||||
import org.elasticsearch.action.OriginalIndices;
|
||||
import org.elasticsearch.action.support.IndicesOptions;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
import org.elasticsearch.search.RescoreDocIds;
|
||||
import org.elasticsearch.search.dfs.AggregatedDfs;
|
||||
import org.elasticsearch.search.internal.ShardSearchRequest;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
|
@ -36,26 +40,43 @@ import java.io.IOException;
|
|||
*/
|
||||
public class ShardFetchSearchRequest extends ShardFetchRequest implements IndicesRequest {
|
||||
|
||||
private OriginalIndices originalIndices;
|
||||
private final OriginalIndices originalIndices;
|
||||
private final ShardSearchRequest shardSearchRequest;
|
||||
private final RescoreDocIds rescoreDocIds;
|
||||
private final AggregatedDfs aggregatedDfs;
|
||||
|
||||
public ShardFetchSearchRequest() {
|
||||
|
||||
}
|
||||
|
||||
public ShardFetchSearchRequest(OriginalIndices originalIndices, SearchContextId id, IntArrayList list, ScoreDoc lastEmittedDoc) {
|
||||
public ShardFetchSearchRequest(OriginalIndices originalIndices, ShardSearchContextId id, ShardSearchRequest shardSearchRequest,
|
||||
IntArrayList list, ScoreDoc lastEmittedDoc, RescoreDocIds rescoreDocIds, AggregatedDfs aggregatedDfs) {
|
||||
super(id, list, lastEmittedDoc);
|
||||
this.originalIndices = originalIndices;
|
||||
this.shardSearchRequest = shardSearchRequest;
|
||||
this.rescoreDocIds = rescoreDocIds;
|
||||
this.aggregatedDfs = aggregatedDfs;
|
||||
}
|
||||
|
||||
public ShardFetchSearchRequest(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
originalIndices = OriginalIndices.readOriginalIndices(in);
|
||||
if (in.getVersion().onOrAfter(Version.V_7_10_0)) {
|
||||
shardSearchRequest = in.readOptionalWriteable(ShardSearchRequest::new);
|
||||
rescoreDocIds = new RescoreDocIds(in);
|
||||
aggregatedDfs = in.readOptionalWriteable(AggregatedDfs::new);
|
||||
} else {
|
||||
shardSearchRequest = null;
|
||||
rescoreDocIds = RescoreDocIds.EMPTY;
|
||||
aggregatedDfs = null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
OriginalIndices.writeOriginalIndices(originalIndices, out);
|
||||
if (out.getVersion().onOrAfter(Version.V_7_10_0)) {
|
||||
out.writeOptionalWriteable(shardSearchRequest);
|
||||
rescoreDocIds.writeTo(out);
|
||||
out.writeOptionalWriteable(aggregatedDfs);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -74,4 +95,18 @@ public class ShardFetchSearchRequest extends ShardFetchRequest implements Indice
|
|||
return originalIndices.indicesOptions();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ShardSearchRequest getShardSearchRequest() {
|
||||
return shardSearchRequest;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RescoreDocIds getRescoreDocIds() {
|
||||
return rescoreDocIds;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AggregatedDfs getAggregatedDfs() {
|
||||
return aggregatedDfs;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -105,7 +105,7 @@ public abstract class FilteredSearchContext extends SearchContext {
|
|||
}
|
||||
|
||||
@Override
|
||||
public SearchContextId id() {
|
||||
public ShardSearchContextId id() {
|
||||
return in.id();
|
||||
}
|
||||
|
||||
|
@ -139,21 +139,11 @@ public abstract class FilteredSearchContext extends SearchContext {
|
|||
return in.queryBoost();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getOriginNanoTime() {
|
||||
return in.getOriginNanoTime();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ScrollContext scrollContext() {
|
||||
return in.scrollContext();
|
||||
}
|
||||
|
||||
@Override
|
||||
public SearchContext scrollContext(ScrollContext scroll) {
|
||||
return in.scrollContext(scroll);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SearchContextAggregations aggregations() {
|
||||
return in.aggregations();
|
||||
|
@ -194,11 +184,6 @@ public abstract class FilteredSearchContext extends SearchContext {
|
|||
return in.rescore();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addRescore(RescoreContext rescore) {
|
||||
in.addRescore(rescore);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasScriptFields() {
|
||||
return in.hasScriptFields();
|
||||
|
@ -450,26 +435,6 @@ public abstract class FilteredSearchContext extends SearchContext {
|
|||
return in.docIdsToLoad(docIdsToLoad, docsIdsToLoadFrom, docsIdsToLoadSize);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void accessed(long accessTime) {
|
||||
in.accessed(accessTime);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long lastAccessTime() {
|
||||
return in.lastAccessTime();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long keepAlive() {
|
||||
return in.keepAlive();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void keepAlive(long keepAlive) {
|
||||
in.keepAlive(keepAlive);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DfsSearchResult dfsResult() {
|
||||
return in.dfsResult();
|
||||
|
@ -552,4 +517,14 @@ public abstract class FilteredSearchContext extends SearchContext {
|
|||
public CollapseContext collapse() {
|
||||
return in.collapse();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addRescore(RescoreContext rescore) {
|
||||
in.addRescore(rescore);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReaderContext readerContext() {
|
||||
return in.readerContext();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -33,21 +33,21 @@ import java.util.Map;
|
|||
|
||||
public class InternalScrollSearchRequest extends TransportRequest {
|
||||
|
||||
private SearchContextId contextId;
|
||||
private ShardSearchContextId contextId;
|
||||
|
||||
private Scroll scroll;
|
||||
|
||||
public InternalScrollSearchRequest() {
|
||||
}
|
||||
|
||||
public InternalScrollSearchRequest(SearchScrollRequest request, SearchContextId contextId) {
|
||||
public InternalScrollSearchRequest(SearchScrollRequest request, ShardSearchContextId contextId) {
|
||||
this.contextId = contextId;
|
||||
this.scroll = request.scroll();
|
||||
}
|
||||
|
||||
public InternalScrollSearchRequest(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
contextId = new SearchContextId(in);
|
||||
contextId = new ShardSearchContextId(in);
|
||||
scroll = in.readOptionalWriteable(Scroll::new);
|
||||
}
|
||||
|
||||
|
@ -58,7 +58,7 @@ public class InternalScrollSearchRequest extends TransportRequest {
|
|||
out.writeOptionalWriteable(scroll);
|
||||
}
|
||||
|
||||
public SearchContextId contextId() {
|
||||
public ShardSearchContextId contextId() {
|
||||
return contextId;
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,112 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search.internal;
|
||||
|
||||
import org.elasticsearch.common.lease.Releasable;
|
||||
import org.elasticsearch.common.lease.Releasables;
|
||||
import org.elasticsearch.index.IndexService;
|
||||
import org.elasticsearch.index.engine.Engine;
|
||||
import org.elasticsearch.index.shard.IndexShard;
|
||||
import org.elasticsearch.search.RescoreDocIds;
|
||||
import org.elasticsearch.search.dfs.AggregatedDfs;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
public class LegacyReaderContext extends ReaderContext {
|
||||
private final ShardSearchRequest shardSearchRequest;
|
||||
private final ScrollContext scrollContext;
|
||||
private AggregatedDfs aggregatedDfs;
|
||||
private RescoreDocIds rescoreDocIds;
|
||||
|
||||
private Engine.Searcher searcher;
|
||||
private Releasable onClose;
|
||||
|
||||
public LegacyReaderContext(long id, IndexService indexService, IndexShard indexShard, Engine.SearcherSupplier reader,
|
||||
ShardSearchRequest shardSearchRequest, long keepAliveInMillis) {
|
||||
super(id, indexService, indexShard, reader, keepAliveInMillis, false);
|
||||
assert shardSearchRequest.readerId() == null;
|
||||
assert shardSearchRequest.keepAlive() == null;
|
||||
this.shardSearchRequest = Objects.requireNonNull(shardSearchRequest);
|
||||
if (shardSearchRequest.scroll() != null) {
|
||||
this.scrollContext = new ScrollContext();
|
||||
} else {
|
||||
this.scrollContext = null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Engine.Searcher acquireSearcher(String source) {
|
||||
if (scrollContext != null && "search".equals(source)) {
|
||||
// Search scroll requests are special, they don't hold indices names so we have
|
||||
// to reuse the searcher created on the request that initialized the scroll.
|
||||
// This ensures that we wrap the searcher's reader with the user's permissions
|
||||
// when they are available.
|
||||
if (searcher == null) {
|
||||
Engine.Searcher delegate = searcherSupplier.acquireSearcher(source);
|
||||
onClose = delegate::close;
|
||||
searcher = new Engine.Searcher(delegate.source(), delegate.getDirectoryReader(),
|
||||
delegate.getSimilarity(), delegate.getQueryCache(), delegate.getQueryCachingPolicy(), () -> {});
|
||||
}
|
||||
return searcher;
|
||||
}
|
||||
return super.acquireSearcher(source);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
void doClose() {
|
||||
Releasables.close(onClose, super::doClose);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ShardSearchRequest getShardSearchRequest(ShardSearchRequest other) {
|
||||
return shardSearchRequest;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ScrollContext scrollContext() {
|
||||
return scrollContext;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AggregatedDfs getAggregatedDfs(AggregatedDfs other) {
|
||||
return aggregatedDfs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setAggregatedDfs(AggregatedDfs aggregatedDfs) {
|
||||
this.aggregatedDfs = aggregatedDfs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RescoreDocIds getRescoreDocIds(RescoreDocIds other) {
|
||||
return rescoreDocIds;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setRescoreDocIds(RescoreDocIds rescoreDocIds) {
|
||||
this.rescoreDocIds = rescoreDocIds;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean singleSession() {
|
||||
return scrollContext == null || scrollContext.scroll == null;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,203 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search.internal;
|
||||
|
||||
import org.elasticsearch.common.UUIDs;
|
||||
import org.elasticsearch.common.lease.Releasable;
|
||||
import org.elasticsearch.common.lease.Releasables;
|
||||
import org.elasticsearch.common.util.concurrent.AbstractRefCounted;
|
||||
import org.elasticsearch.index.IndexService;
|
||||
import org.elasticsearch.index.engine.Engine;
|
||||
import org.elasticsearch.index.shard.IndexShard;
|
||||
import org.elasticsearch.search.RescoreDocIds;
|
||||
import org.elasticsearch.search.dfs.AggregatedDfs;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
/**
|
||||
* Holds a reference to a point in time {@link Engine.Searcher} that will be used to construct {@link SearchContext}.
|
||||
* This class also implements {@link org.elasticsearch.common.util.concurrent.RefCounted} since in some situations like
|
||||
* in {@link org.elasticsearch.search.SearchService} a SearchContext can be closed concurrently due to independent events
|
||||
* ie. when an index gets removed. To prevent accessing closed IndexReader / IndexSearcher instances the SearchContext
|
||||
* can be guarded by a reference count and fail if it's been closed by an external event.
|
||||
*/
|
||||
public class ReaderContext implements Releasable {
|
||||
private final ShardSearchContextId id;
|
||||
private final IndexService indexService;
|
||||
private final IndexShard indexShard;
|
||||
protected final Engine.SearcherSupplier searcherSupplier;
|
||||
private final AtomicBoolean closed = new AtomicBoolean(false);
|
||||
private final boolean singleSession;
|
||||
|
||||
private final AtomicLong keepAlive;
|
||||
private final AtomicLong lastAccessTime;
|
||||
// For reference why we use RefCounted here see https://github.com/elastic/elasticsearch/pull/20095.
|
||||
private final AbstractRefCounted refCounted;
|
||||
|
||||
private final List<Releasable> onCloses = new CopyOnWriteArrayList<>();
|
||||
|
||||
private final long startTimeInNano = System.nanoTime();
|
||||
|
||||
private Map<String, Object> context;
|
||||
|
||||
public ReaderContext(long id,
|
||||
IndexService indexService,
|
||||
IndexShard indexShard,
|
||||
Engine.SearcherSupplier searcherSupplier,
|
||||
long keepAliveInMillis,
|
||||
boolean singleSession) {
|
||||
this.id = new ShardSearchContextId(UUIDs.base64UUID(), id);
|
||||
this.indexService = indexService;
|
||||
this.indexShard = indexShard;
|
||||
this.searcherSupplier = searcherSupplier;
|
||||
this.singleSession = singleSession;
|
||||
this.keepAlive = new AtomicLong(keepAliveInMillis);
|
||||
this.lastAccessTime = new AtomicLong(nowInMillis());
|
||||
this.refCounted = new AbstractRefCounted("reader_context") {
|
||||
@Override
|
||||
protected void closeInternal() {
|
||||
doClose();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private long nowInMillis() {
|
||||
return indexShard.getThreadPool().relativeTimeInMillis();
|
||||
}
|
||||
|
||||
@Override
|
||||
public final void close() {
|
||||
if (closed.compareAndSet(false, true)) {
|
||||
refCounted.decRef();
|
||||
}
|
||||
}
|
||||
|
||||
void doClose() {
|
||||
Releasables.close(Releasables.wrap(onCloses), searcherSupplier);
|
||||
}
|
||||
|
||||
public void addOnClose(Releasable releasable) {
|
||||
onCloses.add(releasable);
|
||||
}
|
||||
|
||||
public ShardSearchContextId id() {
|
||||
return id;
|
||||
}
|
||||
|
||||
public IndexService indexService() {
|
||||
return indexService;
|
||||
}
|
||||
|
||||
public IndexShard indexShard() {
|
||||
return indexShard;
|
||||
}
|
||||
|
||||
public Engine.Searcher acquireSearcher(String source) {
|
||||
return searcherSupplier.acquireSearcher(source);
|
||||
}
|
||||
|
||||
public void keepAlive(long keepAlive) {
|
||||
this.keepAlive.updateAndGet(curr -> Math.max(curr, keepAlive));
|
||||
}
|
||||
|
||||
/**
|
||||
* Marks this reader as being used so its time to live should not be expired.
|
||||
*
|
||||
* @return a releasable to indicate the caller has stopped using this reader
|
||||
*/
|
||||
public Releasable markAsUsed() {
|
||||
refCounted.incRef();
|
||||
return Releasables.releaseOnce(() -> {
|
||||
this.lastAccessTime.updateAndGet(curr -> Math.max(curr, nowInMillis()));
|
||||
refCounted.decRef();
|
||||
});
|
||||
}
|
||||
|
||||
public boolean isExpired() {
|
||||
if (refCounted.refCount() > 1) {
|
||||
return false; // being used by markAsUsed
|
||||
}
|
||||
final long elapsed = nowInMillis() - lastAccessTime.get();
|
||||
return elapsed > keepAlive.get();
|
||||
}
|
||||
|
||||
// BWC
|
||||
public ShardSearchRequest getShardSearchRequest(ShardSearchRequest other) {
|
||||
return Objects.requireNonNull(other);
|
||||
}
|
||||
|
||||
public ScrollContext scrollContext() {
|
||||
return null;
|
||||
}
|
||||
|
||||
public AggregatedDfs getAggregatedDfs(AggregatedDfs other) {
|
||||
return other;
|
||||
}
|
||||
|
||||
public void setAggregatedDfs(AggregatedDfs aggregatedDfs) {
|
||||
|
||||
}
|
||||
|
||||
public RescoreDocIds getRescoreDocIds(RescoreDocIds other) {
|
||||
return Objects.requireNonNull(other);
|
||||
}
|
||||
|
||||
public void setRescoreDocIds(RescoreDocIds rescoreDocIds) {
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns {@code true} for readers that are intended to use in a single query. For readers that are intended
|
||||
* to use in multiple queries (i.e., scroll or readers), we should not release them after the fetch phase
|
||||
* or the query phase with empty results.
|
||||
*/
|
||||
public boolean singleSession() {
|
||||
return singleSession;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the object or <code>null</code> if the given key does not have a
|
||||
* value in the context
|
||||
*/
|
||||
@SuppressWarnings("unchecked") // (T)object
|
||||
public <T> T getFromContext(String key) {
|
||||
return context != null ? (T) context.get(key) : null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Puts the object into the context
|
||||
*/
|
||||
public void putInContext(String key, Object value) {
|
||||
if (context == null) {
|
||||
context = new HashMap<>();
|
||||
}
|
||||
context.put(key, value);
|
||||
}
|
||||
|
||||
public long getStartTimeInNano() {
|
||||
return startTimeInNano;
|
||||
}
|
||||
}
|
|
@ -23,35 +23,10 @@ import org.apache.lucene.search.ScoreDoc;
|
|||
import org.apache.lucene.search.TotalHits;
|
||||
import org.elasticsearch.search.Scroll;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
/** Wrapper around information that needs to stay around when scrolling. */
|
||||
public final class ScrollContext {
|
||||
|
||||
private Map<String, Object> context = null;
|
||||
|
||||
public TotalHits totalHits = null;
|
||||
public float maxScore = Float.NaN;
|
||||
public ScoreDoc lastEmittedDoc;
|
||||
public Scroll scroll;
|
||||
|
||||
/**
|
||||
* Returns the object or <code>null</code> if the given key does not have a
|
||||
* value in the context
|
||||
*/
|
||||
@SuppressWarnings("unchecked") // (T)object
|
||||
public <T> T getFromContext(String key) {
|
||||
return context != null ? (T) context.get(key) : null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Puts the object into the context
|
||||
*/
|
||||
public void putInContext(String key, Object value) {
|
||||
if (context == null) {
|
||||
context = new HashMap<>();
|
||||
}
|
||||
context.put(key, value);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -29,9 +29,6 @@ import org.elasticsearch.common.lease.Releasable;
|
|||
import org.elasticsearch.common.lease.Releasables;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.BigArrays;
|
||||
import org.elasticsearch.common.util.concurrent.AbstractRefCounted;
|
||||
import org.elasticsearch.common.util.concurrent.RefCounted;
|
||||
import org.elasticsearch.common.util.iterable.Iterables;
|
||||
import org.elasticsearch.index.cache.bitset.BitsetFilterCache;
|
||||
import org.elasticsearch.index.fielddata.IndexFieldData;
|
||||
import org.elasticsearch.index.mapper.MappedFieldType;
|
||||
|
@ -41,6 +38,7 @@ import org.elasticsearch.index.query.ParsedQuery;
|
|||
import org.elasticsearch.index.query.QueryShardContext;
|
||||
import org.elasticsearch.index.shard.IndexShard;
|
||||
import org.elasticsearch.index.similarity.SimilarityService;
|
||||
import org.elasticsearch.search.RescoreDocIds;
|
||||
import org.elasticsearch.search.SearchExtBuilder;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.aggregations.SearchContextAggregations;
|
||||
|
@ -61,37 +59,30 @@ import org.elasticsearch.search.rescore.RescoreContext;
|
|||
import org.elasticsearch.search.sort.SortAndFormats;
|
||||
import org.elasticsearch.search.suggest.SuggestionSearchContext;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.EnumMap;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
/**
|
||||
* This class encapsulates the state needed to execute a search. It holds a reference to the
|
||||
* shards point in time snapshot (IndexReader / ContextIndexSearcher) and allows passing on
|
||||
* state from one query / fetch phase to another.
|
||||
*
|
||||
* This class also implements {@link RefCounted} since in some situations like in {@link org.elasticsearch.search.SearchService}
|
||||
* a SearchContext can be closed concurrently due to independent events ie. when an index gets removed. To prevent accessing closed
|
||||
* IndexReader / IndexSearcher instances the SearchContext can be guarded by a reference count and fail if it's been closed by
|
||||
* an external event.
|
||||
*/
|
||||
// For reference why we use RefCounted here see #20095
|
||||
public abstract class SearchContext extends AbstractRefCounted implements Releasable {
|
||||
public abstract class SearchContext implements Releasable {
|
||||
|
||||
public static final int DEFAULT_TERMINATE_AFTER = 0;
|
||||
public static final int TRACK_TOTAL_HITS_ACCURATE = Integer.MAX_VALUE;
|
||||
public static final int TRACK_TOTAL_HITS_DISABLED = -1;
|
||||
public static final int DEFAULT_TRACK_TOTAL_HITS_UP_TO = 10000;
|
||||
|
||||
private Map<Lifetime, List<Releasable>> clearables = null;
|
||||
private final List<Releasable> releasables = new CopyOnWriteArrayList<>();
|
||||
private final AtomicBoolean closed = new AtomicBoolean(false);
|
||||
private InnerHitsContext innerHitsContext;
|
||||
|
||||
protected SearchContext() {
|
||||
super("search_context");
|
||||
}
|
||||
protected SearchContext() {}
|
||||
|
||||
public abstract void setTask(SearchShardTask task);
|
||||
|
||||
|
@ -101,25 +92,15 @@ public abstract class SearchContext extends AbstractRefCounted implements Releas
|
|||
|
||||
@Override
|
||||
public final void close() {
|
||||
if (closed.compareAndSet(false, true)) { // prevent double closing
|
||||
decRef();
|
||||
if (closed.compareAndSet(false, true)) {
|
||||
try {
|
||||
Releasables.close(releasables);
|
||||
} finally {
|
||||
doClose();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected final void closeInternal() {
|
||||
try {
|
||||
clearReleasables(Lifetime.CONTEXT);
|
||||
} finally {
|
||||
doClose();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void alreadyClosed() {
|
||||
throw new IllegalStateException("search context is already closed can't increment refCount current count [" + refCount() + "]");
|
||||
}
|
||||
|
||||
protected abstract void doClose();
|
||||
|
||||
/**
|
||||
|
@ -132,7 +113,7 @@ public abstract class SearchContext extends AbstractRefCounted implements Releas
|
|||
* alias filters, types filters, etc. */
|
||||
public abstract Query buildFilteredQuery(Query query);
|
||||
|
||||
public abstract SearchContextId id();
|
||||
public abstract ShardSearchContextId id();
|
||||
|
||||
public abstract String source();
|
||||
|
||||
|
@ -146,12 +127,8 @@ public abstract class SearchContext extends AbstractRefCounted implements Releas
|
|||
|
||||
public abstract float queryBoost();
|
||||
|
||||
public abstract long getOriginNanoTime();
|
||||
|
||||
public abstract ScrollContext scrollContext();
|
||||
|
||||
public abstract SearchContext scrollContext(ScrollContext scroll);
|
||||
|
||||
public abstract SearchContextAggregations aggregations();
|
||||
|
||||
public abstract SearchContext aggregations(SearchContextAggregations aggregations);
|
||||
|
@ -182,6 +159,36 @@ public abstract class SearchContext extends AbstractRefCounted implements Releas
|
|||
|
||||
public abstract void addRescore(RescoreContext rescore);
|
||||
|
||||
public final RescoreDocIds rescoreDocIds() {
|
||||
final List<RescoreContext> rescore = rescore();
|
||||
if (rescore == null) {
|
||||
return RescoreDocIds.EMPTY;
|
||||
}
|
||||
Map<Integer, Set<Integer>> rescoreDocIds = null;
|
||||
for (int i = 0; i < rescore.size(); i++) {
|
||||
final Set<Integer> docIds = rescore.get(i).getRescoredDocs();
|
||||
if (docIds != null && docIds.isEmpty() == false) {
|
||||
if (rescoreDocIds == null) {
|
||||
rescoreDocIds = new HashMap<>();
|
||||
}
|
||||
rescoreDocIds.put(i, docIds);
|
||||
}
|
||||
}
|
||||
return rescoreDocIds == null ? RescoreDocIds.EMPTY : new RescoreDocIds(rescoreDocIds);
|
||||
}
|
||||
|
||||
public final void assignRescoreDocIds(RescoreDocIds rescoreDocIds) {
|
||||
final List<RescoreContext> rescore = rescore();
|
||||
if (rescore != null) {
|
||||
for (int i = 0; i < rescore.size(); i++) {
|
||||
final Set<Integer> docIds = rescoreDocIds.getId(i);
|
||||
if (docIds != null) {
|
||||
rescore.get(i).setRescoredDocs(docIds);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public abstract boolean hasScriptFields();
|
||||
|
||||
public abstract ScriptFieldsContext scriptFields();
|
||||
|
@ -332,14 +339,6 @@ public abstract class SearchContext extends AbstractRefCounted implements Releas
|
|||
|
||||
public abstract SearchContext docIdsToLoad(int[] docIdsToLoad, int docsIdsToLoadFrom, int docsIdsToLoadSize);
|
||||
|
||||
public abstract void accessed(long accessTime);
|
||||
|
||||
public abstract long lastAccessTime();
|
||||
|
||||
public abstract long keepAlive();
|
||||
|
||||
public abstract void keepAlive(long keepAlive);
|
||||
|
||||
public abstract DfsSearchResult dfsResult();
|
||||
|
||||
public abstract QuerySearchResult queryResult();
|
||||
|
@ -353,36 +352,12 @@ public abstract class SearchContext extends AbstractRefCounted implements Releas
|
|||
*/
|
||||
public abstract Profilers getProfilers();
|
||||
|
||||
/**
|
||||
* Schedule the release of a resource. The time when {@link Releasable#close()} will be called on this object
|
||||
* is function of the provided {@link Lifetime}.
|
||||
*/
|
||||
public void addReleasable(Releasable releasable, Lifetime lifetime) {
|
||||
if (clearables == null) {
|
||||
clearables = new EnumMap<>(Lifetime.class);
|
||||
}
|
||||
List<Releasable> releasables = clearables.get(lifetime);
|
||||
if (releasables == null) {
|
||||
releasables = new ArrayList<>();
|
||||
clearables.put(lifetime, releasables);
|
||||
}
|
||||
releasables.add(releasable);
|
||||
}
|
||||
|
||||
public void clearReleasables(Lifetime lifetime) {
|
||||
if (clearables != null) {
|
||||
List<List<Releasable>>releasables = new ArrayList<>();
|
||||
for (Lifetime lc : Lifetime.values()) {
|
||||
if (lc.compareTo(lifetime) > 0) {
|
||||
break;
|
||||
}
|
||||
List<Releasable> remove = clearables.remove(lc);
|
||||
if (remove != null) {
|
||||
releasables.add(remove);
|
||||
}
|
||||
}
|
||||
Releasables.close(Iterables.flatten(releasables));
|
||||
}
|
||||
/**
|
||||
* Adds a releasable that will be freed when this context is closed.
|
||||
*/
|
||||
public void addReleasable(Releasable releasable) {
|
||||
releasables.add(releasable);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -409,24 +384,6 @@ public abstract class SearchContext extends AbstractRefCounted implements Releas
|
|||
/** Return a view of the additional query collectors that should be run for this context. */
|
||||
public abstract Map<Class<?>, Collector> queryCollectors();
|
||||
|
||||
/**
|
||||
* The life time of an object that is used during search execution.
|
||||
*/
|
||||
public enum Lifetime {
|
||||
/**
|
||||
* This life time is for objects that only live during collection time.
|
||||
*/
|
||||
COLLECTION,
|
||||
/**
|
||||
* This life time is for objects that need to live until the end of the current search phase.
|
||||
*/
|
||||
PHASE,
|
||||
/**
|
||||
* This life time is for objects that need to live until the search context they are attached to is destroyed.
|
||||
*/
|
||||
CONTEXT
|
||||
}
|
||||
|
||||
public abstract QueryShardContext getQueryShardContext();
|
||||
|
||||
@Override
|
||||
|
@ -445,4 +402,6 @@ public abstract class SearchContext extends AbstractRefCounted implements Releas
|
|||
result.append(" query=[").append(query()).append("]");
|
||||
return result.toString();
|
||||
}
|
||||
|
||||
public abstract ReaderContext readerContext();
|
||||
}
|
||||
|
|
|
@ -27,17 +27,16 @@ import org.elasticsearch.common.io.stream.Writeable;
|
|||
import java.io.IOException;
|
||||
import java.util.Objects;
|
||||
|
||||
|
||||
public final class SearchContextId implements Writeable {
|
||||
public final class ShardSearchContextId implements Writeable {
|
||||
private final String readerId;
|
||||
private final long id;
|
||||
|
||||
public SearchContextId(String readerId, long id) {
|
||||
public ShardSearchContextId(String readerId, long id) {
|
||||
this.readerId = Objects.requireNonNull(readerId);
|
||||
this.id = id;
|
||||
}
|
||||
|
||||
public SearchContextId(StreamInput in) throws IOException {
|
||||
public ShardSearchContextId(StreamInput in) throws IOException {
|
||||
this.id = in.readLong();
|
||||
if (in.getVersion().onOrAfter(Version.V_7_7_0)) {
|
||||
this.readerId = in.readString();
|
||||
|
@ -66,7 +65,7 @@ public final class SearchContextId implements Writeable {
|
|||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
SearchContextId other = (SearchContextId) o;
|
||||
ShardSearchContextId other = (ShardSearchContextId) o;
|
||||
return id == other.id && readerId.equals(other.readerId);
|
||||
}
|
||||
|
|
@ -36,6 +36,7 @@ import org.elasticsearch.common.collect.ImmutableOpenMap;
|
|||
import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.query.BoolQueryBuilder;
|
||||
import org.elasticsearch.index.query.MatchNoneQueryBuilder;
|
||||
|
@ -87,6 +88,8 @@ public class ShardSearchRequest extends TransportRequest implements IndicesReque
|
|||
//these are the only mutable fields, as they are subject to rewriting
|
||||
private AliasFilter aliasFilter;
|
||||
private SearchSourceBuilder source;
|
||||
private final ShardSearchContextId readerId;
|
||||
private final TimeValue keepAlive;
|
||||
|
||||
public ShardSearchRequest(OriginalIndices originalIndices,
|
||||
SearchRequest searchRequest,
|
||||
|
@ -97,6 +100,21 @@ public class ShardSearchRequest extends TransportRequest implements IndicesReque
|
|||
long nowInMillis,
|
||||
@Nullable String clusterAlias,
|
||||
String[] indexRoutings) {
|
||||
this(originalIndices, searchRequest, shardId, numberOfShards, aliasFilter,
|
||||
indexBoost, nowInMillis, clusterAlias, indexRoutings, null, null);
|
||||
}
|
||||
|
||||
public ShardSearchRequest(OriginalIndices originalIndices,
|
||||
SearchRequest searchRequest,
|
||||
ShardId shardId,
|
||||
int numberOfShards,
|
||||
AliasFilter aliasFilter,
|
||||
float indexBoost,
|
||||
long nowInMillis,
|
||||
@Nullable String clusterAlias,
|
||||
String[] indexRoutings,
|
||||
ShardSearchContextId readerId,
|
||||
TimeValue keepAlive) {
|
||||
this(originalIndices,
|
||||
shardId,
|
||||
numberOfShards,
|
||||
|
@ -111,7 +129,9 @@ public class ShardSearchRequest extends TransportRequest implements IndicesReque
|
|||
searchRequest.preference(),
|
||||
searchRequest.scroll(),
|
||||
nowInMillis,
|
||||
clusterAlias);
|
||||
clusterAlias,
|
||||
readerId,
|
||||
keepAlive);
|
||||
// If allowPartialSearchResults is unset (ie null), the cluster-level default should have been substituted
|
||||
// at this stage. Any NPEs in the above are therefore an error in request preparation logic.
|
||||
assert searchRequest.allowPartialSearchResults() != null;
|
||||
|
@ -122,7 +142,7 @@ public class ShardSearchRequest extends TransportRequest implements IndicesReque
|
|||
long nowInMillis,
|
||||
AliasFilter aliasFilter) {
|
||||
this(OriginalIndices.NONE, shardId, -1, SearchType.QUERY_THEN_FETCH, null, types,
|
||||
null, aliasFilter, 1.0f, false, Strings.EMPTY_ARRAY, null, null, nowInMillis, null);
|
||||
null, aliasFilter, 1.0f, false, Strings.EMPTY_ARRAY, null, null, nowInMillis, null, null, null);
|
||||
}
|
||||
|
||||
private ShardSearchRequest(OriginalIndices originalIndices,
|
||||
|
@ -139,7 +159,9 @@ public class ShardSearchRequest extends TransportRequest implements IndicesReque
|
|||
String preference,
|
||||
Scroll scroll,
|
||||
long nowInMillis,
|
||||
@Nullable String clusterAlias) {
|
||||
@Nullable String clusterAlias,
|
||||
ShardSearchContextId readerId,
|
||||
TimeValue keepAlive) {
|
||||
this.shardId = shardId;
|
||||
this.numberOfShards = numberOfShards;
|
||||
this.searchType = searchType;
|
||||
|
@ -155,6 +177,9 @@ public class ShardSearchRequest extends TransportRequest implements IndicesReque
|
|||
this.nowInMillis = nowInMillis;
|
||||
this.clusterAlias = clusterAlias;
|
||||
this.originalIndices = originalIndices;
|
||||
this.readerId = readerId;
|
||||
this.keepAlive = keepAlive;
|
||||
assert (readerId != null) == (keepAlive != null);
|
||||
}
|
||||
|
||||
public ShardSearchRequest(StreamInput in) throws IOException {
|
||||
|
@ -191,7 +216,15 @@ public class ShardSearchRequest extends TransportRequest implements IndicesReque
|
|||
canReturnNullResponseIfMatchNoDocs = false;
|
||||
bottomSortValues = null;
|
||||
}
|
||||
if (in.getVersion().onOrAfter(Version.V_7_10_0)) {
|
||||
this.readerId = in.readOptionalWriteable(ShardSearchContextId::new);
|
||||
this.keepAlive = in.readOptionalTimeValue();
|
||||
} else {
|
||||
this.readerId = null;
|
||||
this.keepAlive = null;
|
||||
}
|
||||
originalIndices = OriginalIndices.readOriginalIndices(in);
|
||||
assert (readerId != null) == (keepAlive != null);
|
||||
}
|
||||
|
||||
public ShardSearchRequest(ShardSearchRequest clone) {
|
||||
|
@ -212,6 +245,8 @@ public class ShardSearchRequest extends TransportRequest implements IndicesReque
|
|||
this.canReturnNullResponseIfMatchNoDocs = clone.canReturnNullResponseIfMatchNoDocs;
|
||||
this.bottomSortValues = clone.bottomSortValues;
|
||||
this.originalIndices = clone.originalIndices;
|
||||
this.readerId = clone.readerId;
|
||||
this.keepAlive = clone.keepAlive;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -252,6 +287,10 @@ public class ShardSearchRequest extends TransportRequest implements IndicesReque
|
|||
out.writeBoolean(canReturnNullResponseIfMatchNoDocs);
|
||||
out.writeOptionalWriteable(bottomSortValues);
|
||||
}
|
||||
if (out.getVersion().onOrAfter(Version.V_7_10_0) && asKey == false) {
|
||||
out.writeOptionalWriteable(readerId);
|
||||
out.writeOptionalTimeValue(keepAlive);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -359,6 +398,21 @@ public class ShardSearchRequest extends TransportRequest implements IndicesReque
|
|||
|
||||
private static final ThreadLocal<BytesStreamOutput> scratch = ThreadLocal.withInitial(BytesStreamOutput::new);
|
||||
|
||||
/**
|
||||
* Returns a non-null value if this request should execute using a specific point-in-time reader;
|
||||
* otherwise, using the most up to date point-in-time reader.
|
||||
*/
|
||||
public ShardSearchContextId readerId() {
|
||||
return readerId;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a non-null to specify the time to live of the point-in-time reader that is used to execute this request.
|
||||
*/
|
||||
public TimeValue keepAlive() {
|
||||
return keepAlive;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the cache key for this shard search request, based on its content
|
||||
*/
|
||||
|
|
|
@ -31,7 +31,6 @@ import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
|
|||
import org.elasticsearch.search.fetch.subphase.ScriptFieldsContext;
|
||||
import org.elasticsearch.search.fetch.subphase.highlight.SearchHighlightContext;
|
||||
import org.elasticsearch.search.query.QuerySearchResult;
|
||||
import org.elasticsearch.search.rescore.RescoreContext;
|
||||
import org.elasticsearch.search.sort.SortAndFormats;
|
||||
import org.elasticsearch.search.suggest.SuggestionSearchContext;
|
||||
|
||||
|
@ -87,11 +86,6 @@ public class SubSearchContext extends FilteredSearchContext {
|
|||
throw new UnsupportedOperationException("this context should be read only");
|
||||
}
|
||||
|
||||
@Override
|
||||
public SearchContext scrollContext(ScrollContext scrollContext) {
|
||||
throw new UnsupportedOperationException("Not supported");
|
||||
}
|
||||
|
||||
@Override
|
||||
public SearchContext aggregations(SearchContextAggregations aggregations) {
|
||||
throw new UnsupportedOperationException("Not supported");
|
||||
|
@ -112,11 +106,6 @@ public class SubSearchContext extends FilteredSearchContext {
|
|||
throw new UnsupportedOperationException("Not supported");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addRescore(RescoreContext rescore) {
|
||||
throw new UnsupportedOperationException("Not supported");
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasScriptFields() {
|
||||
return scriptFields != null;
|
||||
|
@ -345,16 +334,6 @@ public class SubSearchContext extends FilteredSearchContext {
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void accessed(long accessTime) {
|
||||
throw new UnsupportedOperationException("Not supported");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void keepAlive(long keepAlive) {
|
||||
throw new UnsupportedOperationException("Not supported");
|
||||
}
|
||||
|
||||
@Override
|
||||
public QuerySearchResult queryResult() {
|
||||
return querySearchResult;
|
||||
|
|
|
@ -348,8 +348,6 @@ public class QueryPhase {
|
|||
throw new QueryPhaseExecutionException(searchContext.shardTarget(), "Time exceeded");
|
||||
}
|
||||
queryResult.searchTimedOut(true);
|
||||
} finally {
|
||||
searchContext.clearReleasables(SearchContext.Lifetime.COLLECTION);
|
||||
}
|
||||
if (searchContext.terminateAfter() != SearchContext.DEFAULT_TERMINATE_AFTER && queryResult.terminatedEarly() == null) {
|
||||
queryResult.terminatedEarly(false);
|
||||
|
@ -404,8 +402,6 @@ public class QueryPhase {
|
|||
throw new QueryPhaseExecutionException(searchContext.shardTarget(), "Time exceeded");
|
||||
}
|
||||
searchContext.queryResult().searchTimedOut(true);
|
||||
} finally {
|
||||
searchContext.clearReleasables(SearchContext.Lifetime.COLLECTION);
|
||||
}
|
||||
return false; // no rescoring when sorting by field
|
||||
}
|
||||
|
|
|
@ -19,15 +19,18 @@
|
|||
|
||||
package org.elasticsearch.search.query;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.IndicesRequest;
|
||||
import org.elasticsearch.action.OriginalIndices;
|
||||
import org.elasticsearch.action.search.SearchShardTask;
|
||||
import org.elasticsearch.action.support.IndicesOptions;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.search.dfs.AggregatedDfs;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchRequest;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.tasks.TaskId;
|
||||
import org.elasticsearch.transport.TransportRequest;
|
||||
|
@ -37,23 +40,29 @@ import java.util.Map;
|
|||
|
||||
public class QuerySearchRequest extends TransportRequest implements IndicesRequest {
|
||||
|
||||
private final SearchContextId contextId;
|
||||
|
||||
private final ShardSearchContextId contextId;
|
||||
private final AggregatedDfs dfs;
|
||||
|
||||
private final OriginalIndices originalIndices;
|
||||
private final ShardSearchRequest shardSearchRequest;
|
||||
|
||||
public QuerySearchRequest(OriginalIndices originalIndices, SearchContextId contextId, AggregatedDfs dfs) {
|
||||
public QuerySearchRequest(OriginalIndices originalIndices, ShardSearchContextId contextId,
|
||||
ShardSearchRequest shardSearchRequest, AggregatedDfs dfs) {
|
||||
this.contextId = contextId;
|
||||
this.dfs = dfs;
|
||||
this.shardSearchRequest = shardSearchRequest;
|
||||
this.originalIndices = originalIndices;
|
||||
}
|
||||
|
||||
public QuerySearchRequest(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
contextId = new SearchContextId(in);
|
||||
contextId = new ShardSearchContextId(in);
|
||||
dfs = new AggregatedDfs(in);
|
||||
originalIndices = OriginalIndices.readOriginalIndices(in);
|
||||
if (in.getVersion().onOrAfter(Version.V_7_10_0)) {
|
||||
this.shardSearchRequest = in.readOptionalWriteable(ShardSearchRequest::new);
|
||||
} else {
|
||||
this.shardSearchRequest = null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -62,9 +71,12 @@ public class QuerySearchRequest extends TransportRequest implements IndicesReque
|
|||
contextId.writeTo(out);
|
||||
dfs.writeTo(out);
|
||||
OriginalIndices.writeOriginalIndices(originalIndices, out);
|
||||
if (out.getVersion().onOrAfter(Version.V_7_10_0)) {
|
||||
out.writeOptionalWriteable(shardSearchRequest);
|
||||
}
|
||||
}
|
||||
|
||||
public SearchContextId contextId() {
|
||||
public ShardSearchContextId contextId() {
|
||||
return contextId;
|
||||
}
|
||||
|
||||
|
@ -72,6 +84,11 @@ public class QuerySearchRequest extends TransportRequest implements IndicesReque
|
|||
return dfs;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public ShardSearchRequest shardSearchRequest() {
|
||||
return shardSearchRequest;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String[] indices() {
|
||||
return originalIndices.indices();
|
||||
|
|
|
@ -33,12 +33,14 @@ import org.elasticsearch.common.io.stream.StreamInput;
|
|||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.lucene.search.TopDocsAndMaxScore;
|
||||
import org.elasticsearch.search.DocValueFormat;
|
||||
import org.elasticsearch.search.RescoreDocIds;
|
||||
import org.elasticsearch.search.SearchPhaseResult;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregations;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchRequest;
|
||||
import org.elasticsearch.search.profile.ProfileShardResult;
|
||||
import org.elasticsearch.search.suggest.Suggest;
|
||||
|
||||
|
@ -82,15 +84,16 @@ public final class QuerySearchResult extends SearchPhaseResult {
|
|||
isNull = false;
|
||||
}
|
||||
if (isNull == false) {
|
||||
SearchContextId id = new SearchContextId(in);
|
||||
ShardSearchContextId id = new ShardSearchContextId(in);
|
||||
readFromWithId(id, in);
|
||||
}
|
||||
}
|
||||
|
||||
public QuerySearchResult(SearchContextId id, SearchShardTarget shardTarget) {
|
||||
this.contextId = id;
|
||||
public QuerySearchResult(ShardSearchContextId contextId, SearchShardTarget shardTarget, ShardSearchRequest shardSearchRequest) {
|
||||
this.contextId = contextId;
|
||||
setSearchShardTarget(shardTarget);
|
||||
isNull = false;
|
||||
setShardSearchRequest(shardSearchRequest);
|
||||
}
|
||||
|
||||
private QuerySearchResult(boolean isNull) {
|
||||
|
@ -315,7 +318,7 @@ public final class QuerySearchResult extends SearchPhaseResult {
|
|||
return hasScoreDocs || hasSuggestHits();
|
||||
}
|
||||
|
||||
public void readFromWithId(SearchContextId id, StreamInput in) throws IOException {
|
||||
public void readFromWithId(ShardSearchContextId id, StreamInput in) throws IOException {
|
||||
this.contextId = id;
|
||||
from = in.readVInt();
|
||||
size = in.readVInt();
|
||||
|
@ -349,12 +352,11 @@ public final class QuerySearchResult extends SearchPhaseResult {
|
|||
terminatedEarly = in.readOptionalBoolean();
|
||||
profileShardResults = in.readOptionalWriteable(ProfileShardResult::new);
|
||||
hasProfileResults = profileShardResults != null;
|
||||
if (in.getVersion().onOrAfter(Version.V_6_0_0_beta1)) {
|
||||
serviceTimeEWMA = in.readZLong();
|
||||
nodeQueueSize = in.readInt();
|
||||
} else {
|
||||
serviceTimeEWMA = -1;
|
||||
nodeQueueSize = -1;
|
||||
serviceTimeEWMA = in.readZLong();
|
||||
nodeQueueSize = in.readInt();
|
||||
if (in.getVersion().onOrAfter(Version.V_7_10_0)) {
|
||||
setShardSearchRequest(in.readOptionalWriteable(ShardSearchRequest::new));
|
||||
setRescoreDocIds(new RescoreDocIds(in));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -425,9 +427,11 @@ public final class QuerySearchResult extends SearchPhaseResult {
|
|||
out.writeBoolean(searchTimedOut);
|
||||
out.writeOptionalBoolean(terminatedEarly);
|
||||
out.writeOptionalWriteable(profileShardResults);
|
||||
if (out.getVersion().onOrAfter(Version.V_6_0_0_beta1)) {
|
||||
out.writeZLong(serviceTimeEWMA);
|
||||
out.writeInt(nodeQueueSize);
|
||||
out.writeZLong(serviceTimeEWMA);
|
||||
out.writeInt(nodeQueueSize);
|
||||
if (out.getVersion().onOrAfter(Version.V_7_10_0)) {
|
||||
out.writeOptionalWriteable(getShardSearchRequest());
|
||||
getRescoreDocIds().writeTo(out);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -63,7 +63,11 @@ public class RescoreContext {
|
|||
}
|
||||
|
||||
public boolean isRescored(int docId) {
|
||||
return rescoredDocs.contains(docId);
|
||||
return rescoredDocs != null && rescoredDocs.contains(docId);
|
||||
}
|
||||
|
||||
public Set<Integer> getRescoredDocs() {
|
||||
return rescoredDocs;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -57,7 +57,7 @@ import org.elasticsearch.script.ScriptException;
|
|||
import org.elasticsearch.search.SearchContextMissingException;
|
||||
import org.elasticsearch.search.SearchParseException;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.transport.RemoteTransportException;
|
||||
|
||||
|
@ -814,7 +814,7 @@ public class ElasticsearchExceptionTests extends ESTestCase {
|
|||
OriginalIndices.NONE)), new ShardSearchFailure(new RepositoryException("repository_g", "Repo"),
|
||||
new SearchShardTarget("node_g", new ShardId(new Index("_index_g", "_uuid_g"), 62), null,
|
||||
OriginalIndices.NONE)), new ShardSearchFailure(
|
||||
new SearchContextMissingException(new SearchContextId(UUIDs.randomBase64UUID(), 0L)), null)
|
||||
new SearchContextMissingException(new ShardSearchContextId(UUIDs.randomBase64UUID(), 0L)), null)
|
||||
};
|
||||
failure = new SearchPhaseExecutionException("phase_g", "G", failureCause, shardFailures);
|
||||
|
||||
|
|
|
@ -81,7 +81,7 @@ import org.elasticsearch.search.SearchException;
|
|||
import org.elasticsearch.search.SearchParseException;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.aggregations.MultiBucketConsumerService;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
import org.elasticsearch.snapshots.Snapshot;
|
||||
import org.elasticsearch.snapshots.SnapshotException;
|
||||
import org.elasticsearch.snapshots.SnapshotId;
|
||||
|
@ -355,7 +355,7 @@ public class ExceptionSerializationTests extends ESTestCase {
|
|||
}
|
||||
|
||||
public void testSearchContextMissingException() throws IOException {
|
||||
SearchContextId contextId = new SearchContextId(UUIDs.randomBase64UUID(), randomLong());
|
||||
ShardSearchContextId contextId = new ShardSearchContextId(UUIDs.randomBase64UUID(), randomLong());
|
||||
Version version = VersionUtils.randomVersion(random());
|
||||
SearchContextMissingException ex = serialize(new SearchContextMissingException(contextId), version);
|
||||
assertThat(ex.contextId().getId(), equalTo(contextId.getId()));
|
||||
|
|
|
@ -35,7 +35,7 @@ import org.elasticsearch.search.SearchPhaseResult;
|
|||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.internal.AliasFilter;
|
||||
import org.elasticsearch.search.internal.InternalSearchResponse;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchRequest;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
|
@ -58,7 +58,7 @@ import static org.hamcrest.Matchers.instanceOf;
|
|||
public class AbstractSearchAsyncActionTests extends ESTestCase {
|
||||
|
||||
private final List<Tuple<String, String>> resolvedNodes = new ArrayList<>();
|
||||
private final Set<SearchContextId> releasedContexts = new CopyOnWriteArraySet<>();
|
||||
private final Set<ShardSearchContextId> releasedContexts = new CopyOnWriteArraySet<>();
|
||||
|
||||
private AbstractSearchAsyncAction<SearchPhaseResult> createAction(SearchRequest request,
|
||||
ArraySearchPhaseResults<SearchPhaseResult> results,
|
||||
|
@ -113,7 +113,7 @@ public class AbstractSearchAsyncActionTests extends ESTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void sendReleaseSearchContext(SearchContextId contextId, Transport.Connection connection,
|
||||
public void sendReleaseSearchContext(ShardSearchContextId contextId, Transport.Connection connection,
|
||||
OriginalIndices originalIndices) {
|
||||
releasedContexts.add(contextId);
|
||||
}
|
||||
|
@ -163,12 +163,11 @@ public class AbstractSearchAsyncActionTests extends ESTestCase {
|
|||
|
||||
public void testBuildSearchResponse() {
|
||||
SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(randomBoolean());
|
||||
ArraySearchPhaseResults<SearchPhaseResult> phaseResults = new ArraySearchPhaseResults<>(10);
|
||||
AbstractSearchAsyncAction<SearchPhaseResult> action = createAction(searchRequest,
|
||||
new ArraySearchPhaseResults<>(10), null, false, new AtomicLong());
|
||||
String scrollId = randomBoolean() ? null : randomAlphaOfLengthBetween(5, 10);
|
||||
phaseResults, null, false, new AtomicLong());
|
||||
InternalSearchResponse internalSearchResponse = InternalSearchResponse.empty();
|
||||
SearchResponse searchResponse = action.buildSearchResponse(internalSearchResponse, scrollId, action.buildShardFailures());
|
||||
assertEquals(scrollId, searchResponse.getScrollId());
|
||||
SearchResponse searchResponse = action.buildSearchResponse(internalSearchResponse, action.buildShardFailures(), null, null);
|
||||
assertSame(searchResponse.getAggregations(), internalSearchResponse.aggregations());
|
||||
assertSame(searchResponse.getSuggest(), internalSearchResponse.suggest());
|
||||
assertSame(searchResponse.getProfileResults(), internalSearchResponse.profile());
|
||||
|
@ -177,14 +176,12 @@ public class AbstractSearchAsyncActionTests extends ESTestCase {
|
|||
|
||||
public void testBuildSearchResponseAllowPartialFailures() {
|
||||
SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true);
|
||||
AbstractSearchAsyncAction<SearchPhaseResult> action = createAction(searchRequest,
|
||||
new ArraySearchPhaseResults<>(10), null, false, new AtomicLong());
|
||||
final ArraySearchPhaseResults<SearchPhaseResult> queryResult = new ArraySearchPhaseResults<>(10);
|
||||
AbstractSearchAsyncAction<SearchPhaseResult> action = createAction(searchRequest, queryResult, null, false, new AtomicLong());
|
||||
action.onShardFailure(0, new SearchShardTarget("node", new ShardId("index", "index-uuid", 0), null, OriginalIndices.NONE),
|
||||
new IllegalArgumentException());
|
||||
String scrollId = randomBoolean() ? null : randomAlphaOfLengthBetween(5, 10);
|
||||
InternalSearchResponse internalSearchResponse = InternalSearchResponse.empty();
|
||||
SearchResponse searchResponse = action.buildSearchResponse(internalSearchResponse, scrollId, action.buildShardFailures());
|
||||
assertEquals(scrollId, searchResponse.getScrollId());
|
||||
SearchResponse searchResponse = action.buildSearchResponse(internalSearchResponse, action.buildShardFailures(), null, null);
|
||||
assertSame(searchResponse.getAggregations(), internalSearchResponse.aggregations());
|
||||
assertSame(searchResponse.getSuggest(), internalSearchResponse.suggest());
|
||||
assertSame(searchResponse.getProfileResults(), internalSearchResponse.profile());
|
||||
|
@ -195,7 +192,7 @@ public class AbstractSearchAsyncActionTests extends ESTestCase {
|
|||
SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(false);
|
||||
AtomicReference<Exception> exception = new AtomicReference<>();
|
||||
ActionListener<SearchResponse> listener = ActionListener.wrap(response -> fail("onResponse should not be called"), exception::set);
|
||||
Set<SearchContextId> requestIds = new HashSet<>();
|
||||
Set<ShardSearchContextId> requestIds = new HashSet<>();
|
||||
List<Tuple<String, String>> nodeLookups = new ArrayList<>();
|
||||
int numFailures = randomIntBetween(1, 5);
|
||||
ArraySearchPhaseResults<SearchPhaseResult> phaseResults = phaseResults(requestIds, nodeLookups, numFailures);
|
||||
|
@ -207,7 +204,7 @@ public class AbstractSearchAsyncActionTests extends ESTestCase {
|
|||
action.onShardFailure(i, new SearchShardTarget(failureNodeId, failureShardId, failureClusterAlias, OriginalIndices.NONE),
|
||||
new IllegalArgumentException());
|
||||
}
|
||||
action.sendSearchResponse(InternalSearchResponse.empty(), randomBoolean() ? null : randomAlphaOfLengthBetween(5, 10));
|
||||
action.sendSearchResponse(InternalSearchResponse.empty(), phaseResults.results);
|
||||
assertThat(exception.get(), instanceOf(SearchPhaseExecutionException.class));
|
||||
SearchPhaseExecutionException searchPhaseExecutionException = (SearchPhaseExecutionException)exception.get();
|
||||
assertEquals(0, searchPhaseExecutionException.getSuppressed().length);
|
||||
|
@ -223,7 +220,7 @@ public class AbstractSearchAsyncActionTests extends ESTestCase {
|
|||
SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(false);
|
||||
AtomicReference<Exception> exception = new AtomicReference<>();
|
||||
ActionListener<SearchResponse> listener = ActionListener.wrap(response -> fail("onResponse should not be called"), exception::set);
|
||||
Set<SearchContextId> requestIds = new HashSet<>();
|
||||
Set<ShardSearchContextId> requestIds = new HashSet<>();
|
||||
List<Tuple<String, String>> nodeLookups = new ArrayList<>();
|
||||
ArraySearchPhaseResults<SearchPhaseResult> phaseResults = phaseResults(requestIds, nodeLookups, 0);
|
||||
AbstractSearchAsyncAction<SearchPhaseResult> action = createAction(searchRequest, phaseResults, listener, false, new AtomicLong());
|
||||
|
@ -266,14 +263,14 @@ public class AbstractSearchAsyncActionTests extends ESTestCase {
|
|||
assertEquals(0, searchPhaseExecutionException.getSuppressed().length);
|
||||
}
|
||||
|
||||
private static ArraySearchPhaseResults<SearchPhaseResult> phaseResults(Set<SearchContextId> contextIds,
|
||||
private static ArraySearchPhaseResults<SearchPhaseResult> phaseResults(Set<ShardSearchContextId> contextIds,
|
||||
List<Tuple<String, String>> nodeLookups,
|
||||
int numFailures) {
|
||||
int numResults = randomIntBetween(1, 10);
|
||||
ArraySearchPhaseResults<SearchPhaseResult> phaseResults = new ArraySearchPhaseResults<>(numResults + numFailures);
|
||||
|
||||
for (int i = 0; i < numResults; i++) {
|
||||
SearchContextId contextId = new SearchContextId(UUIDs.randomBase64UUID(), randomNonNegativeLong());
|
||||
ShardSearchContextId contextId = new ShardSearchContextId(UUIDs.randomBase64UUID(), randomNonNegativeLong());
|
||||
contextIds.add(contextId);
|
||||
SearchPhaseResult phaseResult = new PhaseResult(contextId);
|
||||
String resultClusterAlias = randomBoolean() ? null : randomAlphaOfLengthBetween(5, 10);
|
||||
|
@ -288,7 +285,7 @@ public class AbstractSearchAsyncActionTests extends ESTestCase {
|
|||
}
|
||||
|
||||
private static final class PhaseResult extends SearchPhaseResult {
|
||||
PhaseResult(SearchContextId contextId) {
|
||||
PhaseResult(ShardSearchContextId contextId) {
|
||||
this.contextId = contextId;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,8 +28,9 @@ import org.elasticsearch.common.util.concurrent.AtomicArray;
|
|||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.search.SearchPhaseResult;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.test.VersionUtils;
|
||||
import org.elasticsearch.transport.NodeNotConnectedException;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.elasticsearch.transport.TransportResponse;
|
||||
|
@ -73,14 +74,14 @@ public class ClearScrollControllerTests extends ESTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
Transport.Connection getConnection(String clusterAlias, DiscoveryNode node) {
|
||||
public Transport.Connection getConnection(String clusterAlias, DiscoveryNode node) {
|
||||
return new SearchAsyncActionTests.MockConnection(node);
|
||||
}
|
||||
};
|
||||
ClearScrollRequest clearScrollRequest = new ClearScrollRequest();
|
||||
clearScrollRequest.scrollIds(Arrays.asList("_all"));
|
||||
ClearScrollController controller = new ClearScrollController(clearScrollRequest, listener,
|
||||
nodes, logger, searchTransportService);
|
||||
ClearScrollController controller = new ClearScrollController(
|
||||
clearScrollRequest, listener, nodes, logger, searchTransportService);
|
||||
controller.run();
|
||||
latch.await();
|
||||
assertEquals(3, nodesInvoked.size());
|
||||
|
@ -95,19 +96,19 @@ public class ClearScrollControllerTests extends ESTestCase {
|
|||
DiscoveryNode node3 = new DiscoveryNode("node_3", buildNewFakeTransportAddress(), Version.CURRENT);
|
||||
AtomicArray<SearchPhaseResult> array = new AtomicArray<>(3);
|
||||
SearchAsyncActionTests.TestSearchPhaseResult testSearchPhaseResult1 =
|
||||
new SearchAsyncActionTests.TestSearchPhaseResult(new SearchContextId(UUIDs.randomBase64UUID(), 1), node1);
|
||||
new SearchAsyncActionTests.TestSearchPhaseResult(new ShardSearchContextId(UUIDs.randomBase64UUID(), 1), node1);
|
||||
testSearchPhaseResult1.setSearchShardTarget(new SearchShardTarget("node_1", new ShardId("idx", "uuid1", 2), null, null));
|
||||
SearchAsyncActionTests.TestSearchPhaseResult testSearchPhaseResult2 =
|
||||
new SearchAsyncActionTests.TestSearchPhaseResult(new SearchContextId(UUIDs.randomBase64UUID(), 12), node2);
|
||||
new SearchAsyncActionTests.TestSearchPhaseResult(new ShardSearchContextId(UUIDs.randomBase64UUID(), 12), node2);
|
||||
testSearchPhaseResult2.setSearchShardTarget(new SearchShardTarget("node_2", new ShardId("idy", "uuid2", 42), null, null));
|
||||
SearchAsyncActionTests.TestSearchPhaseResult testSearchPhaseResult3 =
|
||||
new SearchAsyncActionTests.TestSearchPhaseResult(new SearchContextId(UUIDs.randomBase64UUID(), 42), node3);
|
||||
new SearchAsyncActionTests.TestSearchPhaseResult(new ShardSearchContextId(UUIDs.randomBase64UUID(), 42), node3);
|
||||
testSearchPhaseResult3.setSearchShardTarget(new SearchShardTarget("node_3", new ShardId("idy", "uuid2", 43), null, null));
|
||||
array.setOnce(0, testSearchPhaseResult1);
|
||||
array.setOnce(1, testSearchPhaseResult2);
|
||||
array.setOnce(2, testSearchPhaseResult3);
|
||||
AtomicInteger numFreed = new AtomicInteger(0);
|
||||
String scrollId = TransportSearchHelper.buildScrollId(array, randomBoolean());
|
||||
String scrollId = TransportSearchHelper.buildScrollId(array, VersionUtils.randomVersion(random()));
|
||||
DiscoveryNodes nodes = DiscoveryNodes.builder().add(node1).add(node2).add(node3).build();
|
||||
CountDownLatch latch = new CountDownLatch(1);
|
||||
ActionListener<ClearScrollResponse> listener = new LatchedActionListener<>(new ActionListener<ClearScrollResponse>() {
|
||||
|
@ -126,7 +127,7 @@ public class ClearScrollControllerTests extends ESTestCase {
|
|||
SearchTransportService searchTransportService = new SearchTransportService(null, null) {
|
||||
|
||||
@Override
|
||||
public void sendFreeContext(Transport.Connection connection, SearchContextId contextId,
|
||||
public void sendFreeContext(Transport.Connection connection, ShardSearchContextId contextId,
|
||||
ActionListener<SearchFreeContextResponse> listener) {
|
||||
nodesInvoked.add(connection.getNode());
|
||||
boolean freed = randomBoolean();
|
||||
|
@ -138,7 +139,7 @@ public class ClearScrollControllerTests extends ESTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
Transport.Connection getConnection(String clusterAlias, DiscoveryNode node) {
|
||||
public Transport.Connection getConnection(String clusterAlias, DiscoveryNode node) {
|
||||
return new SearchAsyncActionTests.MockConnection(node);
|
||||
}
|
||||
};
|
||||
|
@ -159,13 +160,13 @@ public class ClearScrollControllerTests extends ESTestCase {
|
|||
DiscoveryNode node3 = new DiscoveryNode("node_3", buildNewFakeTransportAddress(), Version.CURRENT);
|
||||
AtomicArray<SearchPhaseResult> array = new AtomicArray<>(3);
|
||||
SearchAsyncActionTests.TestSearchPhaseResult testSearchPhaseResult1 =
|
||||
new SearchAsyncActionTests.TestSearchPhaseResult(new SearchContextId(UUIDs.randomBase64UUID(), 1), node1);
|
||||
new SearchAsyncActionTests.TestSearchPhaseResult(new ShardSearchContextId(UUIDs.randomBase64UUID(), 1), node1);
|
||||
testSearchPhaseResult1.setSearchShardTarget(new SearchShardTarget("node_1", new ShardId("idx", "uuid1", 2), null, null));
|
||||
SearchAsyncActionTests.TestSearchPhaseResult testSearchPhaseResult2 =
|
||||
new SearchAsyncActionTests.TestSearchPhaseResult(new SearchContextId(UUIDs.randomBase64UUID(), 12), node2);
|
||||
new SearchAsyncActionTests.TestSearchPhaseResult(new ShardSearchContextId(UUIDs.randomBase64UUID(), 12), node2);
|
||||
testSearchPhaseResult2.setSearchShardTarget(new SearchShardTarget("node_2", new ShardId("idy", "uuid2", 42), null, null));
|
||||
SearchAsyncActionTests.TestSearchPhaseResult testSearchPhaseResult3 =
|
||||
new SearchAsyncActionTests.TestSearchPhaseResult(new SearchContextId(UUIDs.randomBase64UUID(), 42), node3);
|
||||
new SearchAsyncActionTests.TestSearchPhaseResult(new ShardSearchContextId(UUIDs.randomBase64UUID(), 42), node3);
|
||||
testSearchPhaseResult3.setSearchShardTarget(new SearchShardTarget("node_3", new ShardId("idy", "uuid2", 43), null, null));
|
||||
array.setOnce(0, testSearchPhaseResult1);
|
||||
array.setOnce(1, testSearchPhaseResult2);
|
||||
|
@ -173,7 +174,7 @@ public class ClearScrollControllerTests extends ESTestCase {
|
|||
AtomicInteger numFreed = new AtomicInteger(0);
|
||||
AtomicInteger numFailures = new AtomicInteger(0);
|
||||
AtomicInteger numConnectionFailures = new AtomicInteger(0);
|
||||
String scrollId = TransportSearchHelper.buildScrollId(array, randomBoolean());
|
||||
String scrollId = TransportSearchHelper.buildScrollId(array, VersionUtils.randomVersion(random()));
|
||||
DiscoveryNodes nodes = DiscoveryNodes.builder().add(node1).add(node2).add(node3).build();
|
||||
CountDownLatch latch = new CountDownLatch(1);
|
||||
|
||||
|
@ -197,7 +198,7 @@ public class ClearScrollControllerTests extends ESTestCase {
|
|||
SearchTransportService searchTransportService = new SearchTransportService(null, null) {
|
||||
|
||||
@Override
|
||||
public void sendFreeContext(Transport.Connection connection, SearchContextId contextId,
|
||||
public void sendFreeContext(Transport.Connection connection, ShardSearchContextId contextId,
|
||||
ActionListener<SearchFreeContextResponse> listener) {
|
||||
nodesInvoked.add(connection.getNode());
|
||||
boolean freed = randomBoolean();
|
||||
|
@ -217,7 +218,7 @@ public class ClearScrollControllerTests extends ESTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
Transport.Connection getConnection(String clusterAlias, DiscoveryNode node) {
|
||||
public Transport.Connection getConnection(String clusterAlias, DiscoveryNode node) {
|
||||
if (randomBoolean()) {
|
||||
numFailures.incrementAndGet();
|
||||
numConnectionFailures.incrementAndGet();
|
||||
|
|
|
@ -25,7 +25,7 @@ import org.elasticsearch.index.shard.ShardId;
|
|||
import org.elasticsearch.search.SearchPhaseResult;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.dfs.DfsSearchResult;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
import java.util.ArrayList;
|
||||
|
@ -61,7 +61,8 @@ public class CountedCollectorTests extends ESTestCase {
|
|||
case 1:
|
||||
state.add(1);
|
||||
executor.execute(() -> {
|
||||
DfsSearchResult dfsSearchResult = new DfsSearchResult(new SearchContextId(UUIDs.randomBase64UUID(), shardID), null);
|
||||
DfsSearchResult dfsSearchResult = new DfsSearchResult(
|
||||
new ShardSearchContextId(UUIDs.randomBase64UUID(), shardID), null, null);
|
||||
dfsSearchResult.setShardIndex(shardID);
|
||||
dfsSearchResult.setSearchShardTarget(new SearchShardTarget("foo",
|
||||
new ShardId("bar", "baz", shardID), null, OriginalIndices.NONE));
|
||||
|
|
|
@ -25,7 +25,6 @@ import org.apache.lucene.search.TopDocs;
|
|||
import org.apache.lucene.search.TotalHits;
|
||||
import org.apache.lucene.store.MockDirectoryWrapper;
|
||||
import org.elasticsearch.action.OriginalIndices;
|
||||
import org.elasticsearch.common.UUIDs;
|
||||
import org.elasticsearch.common.lucene.search.TopDocsAndMaxScore;
|
||||
import org.elasticsearch.common.util.concurrent.AtomicArray;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
@ -33,7 +32,7 @@ import org.elasticsearch.search.DocValueFormat;
|
|||
import org.elasticsearch.search.SearchPhaseResult;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.dfs.DfsSearchResult;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
import org.elasticsearch.search.query.QuerySearchRequest;
|
||||
import org.elasticsearch.search.query.QuerySearchResult;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
@ -46,8 +45,8 @@ import java.util.concurrent.atomic.AtomicReference;
|
|||
|
||||
public class DfsQueryPhaseTests extends ESTestCase {
|
||||
|
||||
private static DfsSearchResult newSearchResult(int shardIndex, SearchContextId contextId, SearchShardTarget target) {
|
||||
DfsSearchResult result = new DfsSearchResult(contextId, target);
|
||||
private static DfsSearchResult newSearchResult(int shardIndex, ShardSearchContextId contextId, SearchShardTarget target) {
|
||||
DfsSearchResult result = new DfsSearchResult(contextId, target, null);
|
||||
result.setShardIndex(shardIndex);
|
||||
return result;
|
||||
}
|
||||
|
@ -55,9 +54,9 @@ public class DfsQueryPhaseTests extends ESTestCase {
|
|||
public void testDfsWith2Shards() throws IOException {
|
||||
AtomicArray<DfsSearchResult> results = new AtomicArray<>(2);
|
||||
AtomicReference<AtomicArray<SearchPhaseResult>> responseRef = new AtomicReference<>();
|
||||
results.set(0, newSearchResult(0, new SearchContextId(UUIDs.randomBase64UUID(), 1),
|
||||
results.set(0, newSearchResult(0, new ShardSearchContextId("", 1),
|
||||
new SearchShardTarget("node1", new ShardId("test", "na", 0), null, OriginalIndices.NONE)));
|
||||
results.set(1, newSearchResult(1, new SearchContextId(UUIDs.randomBase64UUID(), 2),
|
||||
results.set(1, newSearchResult(1, new ShardSearchContextId("", 2),
|
||||
new SearchShardTarget("node2", new ShardId("test", "na", 0), null, OriginalIndices.NONE)));
|
||||
results.get(0).termsStatistics(new Term[0], new TermStatistics[0]);
|
||||
results.get(1).termsStatistics(new Term[0], new TermStatistics[0]);
|
||||
|
@ -67,16 +66,16 @@ public class DfsQueryPhaseTests extends ESTestCase {
|
|||
public void sendExecuteQuery(Transport.Connection connection, QuerySearchRequest request, SearchTask task,
|
||||
SearchActionListener<QuerySearchResult> listener) {
|
||||
if (request.contextId().getId() == 1) {
|
||||
QuerySearchResult queryResult = new QuerySearchResult(new SearchContextId(UUIDs.randomBase64UUID(), 123),
|
||||
new SearchShardTarget("node1", new ShardId("test", "na", 0), null, OriginalIndices.NONE));
|
||||
QuerySearchResult queryResult = new QuerySearchResult(new ShardSearchContextId("", 123),
|
||||
new SearchShardTarget("node1", new ShardId("test", "na", 0), null, OriginalIndices.NONE), null);
|
||||
queryResult.topDocs(new TopDocsAndMaxScore(
|
||||
new TopDocs(new TotalHits(1, TotalHits.Relation.EQUAL_TO),
|
||||
new ScoreDoc[] {new ScoreDoc(42, 1.0F)}), 2.0F), new DocValueFormat[0]);
|
||||
queryResult.size(2); // the size of the result set
|
||||
listener.onResponse(queryResult);
|
||||
} else if (request.contextId().getId() == 2) {
|
||||
QuerySearchResult queryResult = new QuerySearchResult(new SearchContextId(UUIDs.randomBase64UUID(), 123),
|
||||
new SearchShardTarget("node2", new ShardId("test", "na", 0), null, OriginalIndices.NONE));
|
||||
QuerySearchResult queryResult = new QuerySearchResult(new ShardSearchContextId("", 123),
|
||||
new SearchShardTarget("node2", new ShardId("test", "na", 0), null, OriginalIndices.NONE), null);
|
||||
queryResult.topDocs(new TopDocsAndMaxScore(
|
||||
new TopDocs(new TotalHits(1, TotalHits.Relation.EQUAL_TO), new ScoreDoc[] {new ScoreDoc(84, 2.0F)}), 2.0F),
|
||||
new DocValueFormat[0]);
|
||||
|
@ -89,7 +88,7 @@ public class DfsQueryPhaseTests extends ESTestCase {
|
|||
};
|
||||
MockSearchPhaseContext mockSearchPhaseContext = new MockSearchPhaseContext(2);
|
||||
mockSearchPhaseContext.searchTransport = searchTransportService;
|
||||
DfsQueryPhase phase = new DfsQueryPhase(results, searchPhaseController(),
|
||||
DfsQueryPhase phase = new DfsQueryPhase(results.asList(), null, searchPhaseController(),
|
||||
(response) -> new SearchPhase("test") {
|
||||
@Override
|
||||
public void run() throws IOException {
|
||||
|
@ -115,10 +114,10 @@ public class DfsQueryPhaseTests extends ESTestCase {
|
|||
public void testDfsWith1ShardFailed() throws IOException {
|
||||
AtomicArray<DfsSearchResult> results = new AtomicArray<>(2);
|
||||
AtomicReference<AtomicArray<SearchPhaseResult>> responseRef = new AtomicReference<>();
|
||||
final SearchContextId ctx1 = new SearchContextId(UUIDs.randomBase64UUID(), 1);
|
||||
final SearchContextId ctx2 = new SearchContextId(UUIDs.randomBase64UUID(), 2);
|
||||
results.set(0, newSearchResult(0, ctx1, new SearchShardTarget("node1", new ShardId("test", "na", 0), null, OriginalIndices.NONE)));
|
||||
results.set(1, newSearchResult(1, ctx2, new SearchShardTarget("node2", new ShardId("test", "na", 0), null, OriginalIndices.NONE)));
|
||||
results.set(0, newSearchResult(0, new ShardSearchContextId("", 1),
|
||||
new SearchShardTarget("node1", new ShardId("test", "na", 0), null, OriginalIndices.NONE)));
|
||||
results.set(1, newSearchResult(1, new ShardSearchContextId("", 2),
|
||||
new SearchShardTarget("node2", new ShardId("test", "na", 0), null, OriginalIndices.NONE)));
|
||||
results.get(0).termsStatistics(new Term[0], new TermStatistics[0]);
|
||||
results.get(1).termsStatistics(new Term[0], new TermStatistics[0]);
|
||||
|
||||
|
@ -127,8 +126,9 @@ public class DfsQueryPhaseTests extends ESTestCase {
|
|||
public void sendExecuteQuery(Transport.Connection connection, QuerySearchRequest request, SearchTask task,
|
||||
SearchActionListener<QuerySearchResult> listener) {
|
||||
if (request.contextId().getId() == 1) {
|
||||
QuerySearchResult queryResult = new QuerySearchResult(new SearchContextId(UUIDs.randomBase64UUID(), 123),
|
||||
new SearchShardTarget("node1", new ShardId("test", "na", 0), null, OriginalIndices.NONE));
|
||||
QuerySearchResult queryResult = new QuerySearchResult(new ShardSearchContextId("", 123),
|
||||
new SearchShardTarget("node1", new ShardId("test", "na", 0),
|
||||
null, OriginalIndices.NONE), null);
|
||||
queryResult.topDocs(new TopDocsAndMaxScore(new TopDocs(
|
||||
new TotalHits(1, TotalHits.Relation.EQUAL_TO),
|
||||
new ScoreDoc[] {new ScoreDoc(42, 1.0F)}), 2.0F), new DocValueFormat[0]);
|
||||
|
@ -143,7 +143,7 @@ public class DfsQueryPhaseTests extends ESTestCase {
|
|||
};
|
||||
MockSearchPhaseContext mockSearchPhaseContext = new MockSearchPhaseContext(2);
|
||||
mockSearchPhaseContext.searchTransport = searchTransportService;
|
||||
DfsQueryPhase phase = new DfsQueryPhase(results, searchPhaseController(),
|
||||
DfsQueryPhase phase = new DfsQueryPhase(results.asList(), null, searchPhaseController(),
|
||||
(response) -> new SearchPhase("test") {
|
||||
@Override
|
||||
public void run() throws IOException {
|
||||
|
@ -164,7 +164,7 @@ public class DfsQueryPhaseTests extends ESTestCase {
|
|||
assertEquals(1, mockSearchPhaseContext.failures.size());
|
||||
assertTrue(mockSearchPhaseContext.failures.get(0).getCause() instanceof MockDirectoryWrapper.FakeIOException);
|
||||
assertEquals(1, mockSearchPhaseContext.releasedSearchContexts.size());
|
||||
assertTrue(mockSearchPhaseContext.releasedSearchContexts.contains(ctx2));
|
||||
assertTrue(mockSearchPhaseContext.releasedSearchContexts.contains(new ShardSearchContextId("", 2L)));
|
||||
assertNull(responseRef.get().get(1));
|
||||
}
|
||||
|
||||
|
@ -172,9 +172,9 @@ public class DfsQueryPhaseTests extends ESTestCase {
|
|||
public void testFailPhaseOnException() throws IOException {
|
||||
AtomicArray<DfsSearchResult> results = new AtomicArray<>(2);
|
||||
AtomicReference<AtomicArray<SearchPhaseResult>> responseRef = new AtomicReference<>();
|
||||
results.set(0, newSearchResult(0, new SearchContextId(UUIDs.randomBase64UUID(), 1),
|
||||
results.set(0, newSearchResult(0, new ShardSearchContextId("", 1),
|
||||
new SearchShardTarget("node1", new ShardId("test", "na", 0), null, OriginalIndices.NONE)));
|
||||
results.set(1, newSearchResult(1, new SearchContextId(UUIDs.randomBase64UUID(), 2),
|
||||
results.set(1, newSearchResult(1, new ShardSearchContextId("", 2),
|
||||
new SearchShardTarget("node2", new ShardId("test", "na", 0), null, OriginalIndices.NONE)));
|
||||
results.get(0).termsStatistics(new Term[0], new TermStatistics[0]);
|
||||
results.get(1).termsStatistics(new Term[0], new TermStatistics[0]);
|
||||
|
@ -184,8 +184,8 @@ public class DfsQueryPhaseTests extends ESTestCase {
|
|||
public void sendExecuteQuery(Transport.Connection connection, QuerySearchRequest request, SearchTask task,
|
||||
SearchActionListener<QuerySearchResult> listener) {
|
||||
if (request.contextId().getId() == 1) {
|
||||
QuerySearchResult queryResult = new QuerySearchResult(new SearchContextId(UUIDs.randomBase64UUID(), 123),
|
||||
new SearchShardTarget("node1", new ShardId("test", "na", 0), null, OriginalIndices.NONE));
|
||||
QuerySearchResult queryResult = new QuerySearchResult(new ShardSearchContextId("", 123),
|
||||
new SearchShardTarget("node1", new ShardId("test", "na", 0), null, OriginalIndices.NONE), null);
|
||||
queryResult.topDocs(new TopDocsAndMaxScore(
|
||||
new TopDocs(new TotalHits(1, TotalHits.Relation.EQUAL_TO),
|
||||
new ScoreDoc[] {new ScoreDoc(42, 1.0F)}), 2.0F), new DocValueFormat[0]);
|
||||
|
@ -200,7 +200,7 @@ public class DfsQueryPhaseTests extends ESTestCase {
|
|||
};
|
||||
MockSearchPhaseContext mockSearchPhaseContext = new MockSearchPhaseContext(2);
|
||||
mockSearchPhaseContext.searchTransport = searchTransportService;
|
||||
DfsQueryPhase phase = new DfsQueryPhase(results, searchPhaseController(),
|
||||
DfsQueryPhase phase = new DfsQueryPhase(results.asList(), null, searchPhaseController(),
|
||||
(response) -> new SearchPhase("test") {
|
||||
@Override
|
||||
public void run() throws IOException {
|
||||
|
|
|
@ -23,7 +23,6 @@ import org.apache.lucene.search.TopDocs;
|
|||
import org.apache.lucene.search.TotalHits;
|
||||
import org.apache.lucene.store.MockDirectoryWrapper;
|
||||
import org.elasticsearch.action.OriginalIndices;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.common.UUIDs;
|
||||
import org.elasticsearch.common.lucene.search.TopDocsAndMaxScore;
|
||||
import org.elasticsearch.common.util.concurrent.EsExecutors;
|
||||
|
@ -35,7 +34,7 @@ import org.elasticsearch.search.SearchShardTarget;
|
|||
import org.elasticsearch.search.fetch.FetchSearchResult;
|
||||
import org.elasticsearch.search.fetch.QueryFetchSearchResult;
|
||||
import org.elasticsearch.search.fetch.ShardFetchSearchRequest;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
import org.elasticsearch.search.query.QuerySearchResult;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.test.InternalAggregationTestCase;
|
||||
|
@ -73,7 +72,7 @@ public class FetchSearchPhaseTests extends ESTestCase {
|
|||
numHits = 0;
|
||||
}
|
||||
|
||||
FetchSearchPhase phase = new FetchSearchPhase(results, controller, mockSearchPhaseContext, ClusterState.EMPTY_STATE,
|
||||
FetchSearchPhase phase = new FetchSearchPhase(results, controller, null, mockSearchPhaseContext,
|
||||
(searchResponse, scrollId) -> new SearchPhase("test") {
|
||||
@Override
|
||||
public void run() {
|
||||
|
@ -99,18 +98,18 @@ public class FetchSearchPhaseTests extends ESTestCase {
|
|||
QueryPhaseResultConsumer results = controller.newSearchPhaseResults(EsExecutors.newDirectExecutorService(),
|
||||
NOOP, mockSearchPhaseContext.getRequest(), 2, exc -> {});
|
||||
int resultSetSize = randomIntBetween(2, 10);
|
||||
final SearchContextId ctx1 = new SearchContextId(UUIDs.randomBase64UUID(), 123);
|
||||
QuerySearchResult queryResult = new QuerySearchResult(ctx1,
|
||||
new SearchShardTarget("node1", new ShardId("test", "na", 0), null, OriginalIndices.NONE));
|
||||
ShardSearchContextId ctx1 = new ShardSearchContextId(UUIDs.base64UUID(), 123);
|
||||
QuerySearchResult queryResult = new QuerySearchResult(ctx1, new SearchShardTarget("node1", new ShardId("test", "na", 0),
|
||||
null, OriginalIndices.NONE), null);
|
||||
queryResult.topDocs(new TopDocsAndMaxScore(new TopDocs(new TotalHits(1, TotalHits.Relation.EQUAL_TO),
|
||||
new ScoreDoc[] {new ScoreDoc(42, 1.0F)}), 2.0F), new DocValueFormat[0]);
|
||||
queryResult.size(resultSetSize); // the size of the result set
|
||||
queryResult.setShardIndex(0);
|
||||
results.consumeResult(queryResult, () -> {});
|
||||
|
||||
final SearchContextId ctx2 = new SearchContextId(UUIDs.randomBase64UUID(), 312);
|
||||
queryResult = new QuerySearchResult(ctx2,
|
||||
new SearchShardTarget("node2", new ShardId("test", "na", 1), null, OriginalIndices.NONE));
|
||||
final ShardSearchContextId ctx2 = new ShardSearchContextId(UUIDs.base64UUID(), 321);
|
||||
queryResult = new QuerySearchResult(
|
||||
ctx2, new SearchShardTarget("node2", new ShardId("test", "na", 1), null, OriginalIndices.NONE), null);
|
||||
queryResult.topDocs(new TopDocsAndMaxScore(new TopDocs(new TotalHits(1, TotalHits.Relation.EQUAL_TO),
|
||||
new ScoreDoc[] {new ScoreDoc(84, 2.0F)}), 2.0F), new DocValueFormat[0]);
|
||||
queryResult.size(resultSetSize);
|
||||
|
@ -133,7 +132,7 @@ public class FetchSearchPhaseTests extends ESTestCase {
|
|||
listener.onResponse(fetchResult);
|
||||
}
|
||||
};
|
||||
FetchSearchPhase phase = new FetchSearchPhase(results, controller, mockSearchPhaseContext, ClusterState.EMPTY_STATE,
|
||||
FetchSearchPhase phase = new FetchSearchPhase(results, controller, null, mockSearchPhaseContext,
|
||||
(searchResponse, scrollId) -> new SearchPhase("test") {
|
||||
@Override
|
||||
public void run() {
|
||||
|
@ -160,18 +159,17 @@ public class FetchSearchPhaseTests extends ESTestCase {
|
|||
QueryPhaseResultConsumer results = controller.newSearchPhaseResults(EsExecutors.newDirectExecutorService(),
|
||||
NOOP, mockSearchPhaseContext.getRequest(), 2, exc -> {});
|
||||
int resultSetSize = randomIntBetween(2, 10);
|
||||
SearchContextId ctx1 = new SearchContextId(UUIDs.randomBase64UUID(), 123);
|
||||
QuerySearchResult queryResult = new QuerySearchResult(ctx1,
|
||||
new SearchShardTarget("node1", new ShardId("test", "na", 0), null, OriginalIndices.NONE));
|
||||
final ShardSearchContextId ctx = new ShardSearchContextId(UUIDs.base64UUID(), 123);
|
||||
QuerySearchResult queryResult = new QuerySearchResult(ctx,
|
||||
new SearchShardTarget("node1", new ShardId("test", "na", 0), null, OriginalIndices.NONE), null);
|
||||
queryResult.topDocs(new TopDocsAndMaxScore(new TopDocs(new TotalHits(1, TotalHits.Relation.EQUAL_TO),
|
||||
new ScoreDoc[] {new ScoreDoc(42, 1.0F)}), 2.0F), new DocValueFormat[0]);
|
||||
queryResult.size(resultSetSize); // the size of the result set
|
||||
queryResult.setShardIndex(0);
|
||||
results.consumeResult(queryResult, () -> {});
|
||||
|
||||
SearchContextId ctx2 = new SearchContextId(UUIDs.randomBase64UUID(), 321);
|
||||
queryResult = new QuerySearchResult(ctx2,
|
||||
new SearchShardTarget("node2", new ShardId("test", "na", 1), null, OriginalIndices.NONE));
|
||||
queryResult = new QuerySearchResult(new ShardSearchContextId("", 321),
|
||||
new SearchShardTarget("node2", new ShardId("test", "na", 1), null, OriginalIndices.NONE), null);
|
||||
queryResult.topDocs(new TopDocsAndMaxScore(new TopDocs(new TotalHits(1, TotalHits.Relation.EQUAL_TO),
|
||||
new ScoreDoc[] {new ScoreDoc(84, 2.0F)}), 2.0F), new DocValueFormat[0]);
|
||||
queryResult.size(resultSetSize);
|
||||
|
@ -193,7 +191,7 @@ public class FetchSearchPhaseTests extends ESTestCase {
|
|||
|
||||
}
|
||||
};
|
||||
FetchSearchPhase phase = new FetchSearchPhase(results, controller, mockSearchPhaseContext, ClusterState.EMPTY_STATE,
|
||||
FetchSearchPhase phase = new FetchSearchPhase(results, controller, null, mockSearchPhaseContext,
|
||||
(searchResponse, scrollId) -> new SearchPhase("test") {
|
||||
@Override
|
||||
public void run() {
|
||||
|
@ -212,7 +210,7 @@ public class FetchSearchPhaseTests extends ESTestCase {
|
|||
assertEquals(1, searchResponse.getShardFailures().length);
|
||||
assertTrue(searchResponse.getShardFailures()[0].getCause() instanceof MockDirectoryWrapper.FakeIOException);
|
||||
assertEquals(1, mockSearchPhaseContext.releasedSearchContexts.size());
|
||||
assertTrue(mockSearchPhaseContext.releasedSearchContexts.contains(ctx1));
|
||||
assertTrue(mockSearchPhaseContext.releasedSearchContexts.contains(ctx));
|
||||
}
|
||||
|
||||
public void testFetchDocsConcurrently() throws InterruptedException {
|
||||
|
@ -225,8 +223,8 @@ public class FetchSearchPhaseTests extends ESTestCase {
|
|||
QueryPhaseResultConsumer results = controller.newSearchPhaseResults(EsExecutors.newDirectExecutorService(), NOOP,
|
||||
mockSearchPhaseContext.getRequest(), numHits, exc -> {});
|
||||
for (int i = 0; i < numHits; i++) {
|
||||
QuerySearchResult queryResult = new QuerySearchResult(new SearchContextId("", i),
|
||||
new SearchShardTarget("node1", new ShardId("test", "na", 0), null, OriginalIndices.NONE));
|
||||
QuerySearchResult queryResult = new QuerySearchResult(new ShardSearchContextId("", i),
|
||||
new SearchShardTarget("node1", new ShardId("test", "na", 0), null, OriginalIndices.NONE), null);
|
||||
queryResult.topDocs(new TopDocsAndMaxScore(new TopDocs(new TotalHits(1, TotalHits.Relation.EQUAL_TO),
|
||||
new ScoreDoc[] {new ScoreDoc(i+1, i)}), i), new DocValueFormat[0]);
|
||||
queryResult.size(resultSetSize); // the size of the result set
|
||||
|
@ -239,14 +237,14 @@ public class FetchSearchPhaseTests extends ESTestCase {
|
|||
SearchActionListener<FetchSearchResult> listener) {
|
||||
new Thread(() -> {
|
||||
FetchSearchResult fetchResult = new FetchSearchResult();
|
||||
fetchResult.hits(new SearchHits(new SearchHit[]{new SearchHit((int) (request.contextId().getId() + 1))},
|
||||
fetchResult.hits(new SearchHits(new SearchHit[] {new SearchHit((int) (request.contextId().getId()+1))},
|
||||
new TotalHits(1, TotalHits.Relation.EQUAL_TO), 100F));
|
||||
listener.onResponse(fetchResult);
|
||||
}).start();
|
||||
}
|
||||
};
|
||||
CountDownLatch latch = new CountDownLatch(1);
|
||||
FetchSearchPhase phase = new FetchSearchPhase(results, controller, mockSearchPhaseContext, ClusterState.EMPTY_STATE,
|
||||
FetchSearchPhase phase = new FetchSearchPhase(results, controller, null, mockSearchPhaseContext,
|
||||
(searchResponse, scrollId) -> new SearchPhase("test") {
|
||||
@Override
|
||||
public void run() {
|
||||
|
@ -283,16 +281,17 @@ public class FetchSearchPhaseTests extends ESTestCase {
|
|||
controller.newSearchPhaseResults(EsExecutors.newDirectExecutorService(),
|
||||
NOOP, mockSearchPhaseContext.getRequest(), 2, exc -> {});
|
||||
int resultSetSize = randomIntBetween(2, 10);
|
||||
QuerySearchResult queryResult = new QuerySearchResult(new SearchContextId("", 123),
|
||||
new SearchShardTarget("node1", new ShardId("test", "na", 0), null, OriginalIndices.NONE));
|
||||
QuerySearchResult queryResult = new QuerySearchResult(new ShardSearchContextId("", 123),
|
||||
new SearchShardTarget("node1", new ShardId("test", "na", 0),
|
||||
null, OriginalIndices.NONE), null);
|
||||
queryResult.topDocs(new TopDocsAndMaxScore(new TopDocs(new TotalHits(1, TotalHits.Relation.EQUAL_TO),
|
||||
new ScoreDoc[] {new ScoreDoc(42, 1.0F)}), 2.0F), new DocValueFormat[0]);
|
||||
queryResult.size(resultSetSize); // the size of the result set
|
||||
queryResult.setShardIndex(0);
|
||||
results.consumeResult(queryResult, () -> {});
|
||||
|
||||
queryResult = new QuerySearchResult(new SearchContextId("", 321),
|
||||
new SearchShardTarget("node2", new ShardId("test", "na", 1), null, OriginalIndices.NONE));
|
||||
queryResult = new QuerySearchResult(new ShardSearchContextId("", 321),
|
||||
new SearchShardTarget("node2", new ShardId("test", "na", 1), null, OriginalIndices.NONE), null);
|
||||
queryResult.topDocs(new TopDocsAndMaxScore(new TopDocs(new TotalHits(1, TotalHits.Relation.EQUAL_TO),
|
||||
new ScoreDoc[] {new ScoreDoc(84, 2.0F)}), 2.0F), new DocValueFormat[0]);
|
||||
queryResult.size(resultSetSize);
|
||||
|
@ -311,14 +310,14 @@ public class FetchSearchPhaseTests extends ESTestCase {
|
|||
fetchResult.hits(new SearchHits(new SearchHit[] {new SearchHit(84)},
|
||||
new TotalHits(1, TotalHits.Relation.EQUAL_TO), 2.0F));
|
||||
} else {
|
||||
assertEquals(request.contextId().getId(), 123);
|
||||
assertEquals(request, 123);
|
||||
fetchResult.hits(new SearchHits(new SearchHit[] {new SearchHit(42)},
|
||||
new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1.0F));
|
||||
}
|
||||
listener.onResponse(fetchResult);
|
||||
}
|
||||
};
|
||||
FetchSearchPhase phase = new FetchSearchPhase(results, controller, mockSearchPhaseContext, ClusterState.EMPTY_STATE,
|
||||
FetchSearchPhase phase = new FetchSearchPhase(results, controller, null, mockSearchPhaseContext,
|
||||
(searchResponse, scrollId) -> new SearchPhase("test") {
|
||||
@Override
|
||||
public void run() {
|
||||
|
@ -340,18 +339,18 @@ public class FetchSearchPhaseTests extends ESTestCase {
|
|||
QueryPhaseResultConsumer results = controller.newSearchPhaseResults(EsExecutors.newDirectExecutorService(),
|
||||
NOOP, mockSearchPhaseContext.getRequest(), 2, exc -> {});
|
||||
int resultSetSize = 1;
|
||||
SearchContextId ctx1 = new SearchContextId(UUIDs.randomBase64UUID(), 123);
|
||||
final ShardSearchContextId ctx1 = new ShardSearchContextId(UUIDs.base64UUID(), 123);
|
||||
QuerySearchResult queryResult = new QuerySearchResult(ctx1,
|
||||
new SearchShardTarget("node1", new ShardId("test", "na", 0), null, OriginalIndices.NONE));
|
||||
new SearchShardTarget("node1", new ShardId("test", "na", 0), null, OriginalIndices.NONE), null);
|
||||
queryResult.topDocs(new TopDocsAndMaxScore(new TopDocs(new TotalHits(1, TotalHits.Relation.EQUAL_TO),
|
||||
new ScoreDoc[] {new ScoreDoc(42, 1.0F)}), 2.0F), new DocValueFormat[0]);
|
||||
queryResult.size(resultSetSize); // the size of the result set
|
||||
queryResult.setShardIndex(0);
|
||||
results.consumeResult(queryResult, () -> {});
|
||||
|
||||
SearchContextId ctx2 = new SearchContextId(UUIDs.randomBase64UUID(), 321);
|
||||
final ShardSearchContextId ctx2 = new ShardSearchContextId(UUIDs.base64UUID(), 321);
|
||||
queryResult = new QuerySearchResult(ctx2,
|
||||
new SearchShardTarget("node2", new ShardId("test", "na", 1), null, OriginalIndices.NONE));
|
||||
new SearchShardTarget("node2", new ShardId("test", "na", 1), null, OriginalIndices.NONE), null);
|
||||
queryResult.topDocs(new TopDocsAndMaxScore(new TopDocs(new TotalHits(1, TotalHits.Relation.EQUAL_TO),
|
||||
new ScoreDoc[] {new ScoreDoc(84, 2.0F)}), 2.0F), new DocValueFormat[0]);
|
||||
queryResult.size(resultSetSize);
|
||||
|
@ -363,7 +362,7 @@ public class FetchSearchPhaseTests extends ESTestCase {
|
|||
public void sendExecuteFetch(Transport.Connection connection, ShardFetchSearchRequest request, SearchTask task,
|
||||
SearchActionListener<FetchSearchResult> listener) {
|
||||
FetchSearchResult fetchResult = new FetchSearchResult();
|
||||
if (request.contextId().equals(ctx2)) {
|
||||
if (request.contextId().getId() == 321) {
|
||||
fetchResult.hits(new SearchHits(new SearchHit[] {new SearchHit(84)},
|
||||
new TotalHits(1, TotalHits.Relation.EQUAL_TO), 2.0F));
|
||||
} else {
|
||||
|
@ -372,7 +371,7 @@ public class FetchSearchPhaseTests extends ESTestCase {
|
|||
listener.onResponse(fetchResult);
|
||||
}
|
||||
};
|
||||
FetchSearchPhase phase = new FetchSearchPhase(results, controller, mockSearchPhaseContext, ClusterState.EMPTY_STATE,
|
||||
FetchSearchPhase phase = new FetchSearchPhase(results, controller, null, mockSearchPhaseContext,
|
||||
(searchResponse, scrollId) -> new SearchPhase("test") {
|
||||
@Override
|
||||
public void run() {
|
||||
|
|
|
@ -20,11 +20,14 @@ package org.elasticsearch.action.search;
|
|||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.OriginalIndices;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.util.concurrent.AtomicArray;
|
||||
import org.elasticsearch.search.SearchPhaseResult;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.internal.InternalSearchResponse;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchRequest;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.junit.Assert;
|
||||
|
@ -47,7 +50,7 @@ public final class MockSearchPhaseContext implements SearchPhaseContext {
|
|||
final AtomicInteger numSuccess;
|
||||
final List<ShardSearchFailure> failures = Collections.synchronizedList(new ArrayList<>());
|
||||
SearchTransportService searchTransport;
|
||||
final Set<SearchContextId> releasedSearchContexts = new HashSet<>();
|
||||
final Set<ShardSearchContextId> releasedSearchContexts = new HashSet<>();
|
||||
final SearchRequest searchRequest = new SearchRequest();
|
||||
final AtomicReference<SearchResponse> searchResponse = new AtomicReference<>();
|
||||
|
||||
|
@ -83,9 +86,12 @@ public final class MockSearchPhaseContext implements SearchPhaseContext {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void sendSearchResponse(InternalSearchResponse internalSearchResponse, String scrollId) {
|
||||
public void sendSearchResponse(InternalSearchResponse internalSearchResponse, AtomicArray<SearchPhaseResult> queryResults) {
|
||||
String scrollId = getRequest().scroll() != null ? TransportSearchHelper.buildScrollId(queryResults, Version.CURRENT) : null;
|
||||
String searchContextId =
|
||||
getRequest().pointInTimeBuilder() != null ? TransportSearchHelper.buildScrollId(queryResults, Version.CURRENT) : null;
|
||||
searchResponse.set(new SearchResponse(internalSearchResponse, scrollId, numShards, numSuccess.get(), 0, 0,
|
||||
failures.toArray(ShardSearchFailure.EMPTY_ARRAY), SearchResponse.Clusters.EMPTY));
|
||||
failures.toArray(ShardSearchFailure.EMPTY_ARRAY), SearchResponse.Clusters.EMPTY, searchContextId));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -136,7 +142,7 @@ public final class MockSearchPhaseContext implements SearchPhaseContext {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void sendReleaseSearchContext(SearchContextId contextId, Transport.Connection connection, OriginalIndices originalIndices) {
|
||||
public void sendReleaseSearchContext(ShardSearchContextId contextId, Transport.Connection connection, OriginalIndices originalIndices) {
|
||||
releasedSearchContexts.add(contextId);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -49,7 +49,7 @@ public class MultiSearchResponseTests extends AbstractXContentTestCase<MultiSear
|
|||
SearchResponse.Clusters clusters = SearchResponseTests.randomClusters();
|
||||
InternalSearchResponse internalSearchResponse = InternalSearchResponse.empty();
|
||||
SearchResponse searchResponse = new SearchResponse(internalSearchResponse, null, totalShards,
|
||||
successfulShards, skippedShards, tookInMillis, ShardSearchFailure.EMPTY_ARRAY, clusters);
|
||||
successfulShards, skippedShards, tookInMillis, ShardSearchFailure.EMPTY_ARRAY, clusters, null);
|
||||
items[i] = new MultiSearchResponse.Item(searchResponse, null);
|
||||
}
|
||||
return new MultiSearchResponse(items, randomNonNegativeLong());
|
||||
|
@ -68,7 +68,7 @@ public class MultiSearchResponseTests extends AbstractXContentTestCase<MultiSear
|
|||
SearchResponse.Clusters clusters = SearchResponseTests.randomClusters();
|
||||
InternalSearchResponse internalSearchResponse = InternalSearchResponse.empty();
|
||||
SearchResponse searchResponse = new SearchResponse(internalSearchResponse, null, totalShards,
|
||||
successfulShards, skippedShards, tookInMillis, ShardSearchFailure.EMPTY_ARRAY, clusters);
|
||||
successfulShards, skippedShards, tookInMillis, ShardSearchFailure.EMPTY_ARRAY, clusters, null);
|
||||
items[i] = new MultiSearchResponse.Item(searchResponse, null);
|
||||
} else {
|
||||
items[i] = new MultiSearchResponse.Item(null, new ElasticsearchException("an error"));
|
||||
|
@ -81,7 +81,7 @@ public class MultiSearchResponseTests extends AbstractXContentTestCase<MultiSear
|
|||
protected MultiSearchResponse doParseInstance(XContentParser parser) throws IOException {
|
||||
return MultiSearchResponse.fromXContext(parser);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
protected void assertEqualInstances(MultiSearchResponse expected, MultiSearchResponse actual) {
|
||||
assertThat(actual.getTook(), equalTo(expected.getTook()));
|
||||
|
@ -106,7 +106,7 @@ public class MultiSearchResponseTests extends AbstractXContentTestCase<MultiSear
|
|||
|
||||
protected Predicate<String> getRandomFieldsExcludeFilterWhenResultHasErrors() {
|
||||
return field -> field.startsWith("responses");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test parsing {@link MultiSearchResponse} with inner failures as they don't support asserting on xcontent equivalence, given that
|
||||
|
@ -123,6 +123,6 @@ public class MultiSearchResponseTests extends AbstractXContentTestCase<MultiSear
|
|||
AbstractXContentTestCase.testFromXContent(NUMBER_OF_TEST_RUNS, instanceSupplier, supportsUnknownFields, Strings.EMPTY_ARRAY,
|
||||
getRandomFieldsExcludeFilterWhenResultHasErrors(), this::createParser, this::doParseInstance,
|
||||
this::assertEqualInstances, assertToXContentEquivalence, ToXContent.EMPTY_PARAMS);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -35,7 +35,7 @@ import org.elasticsearch.index.shard.ShardId;
|
|||
import org.elasticsearch.search.SearchPhaseResult;
|
||||
import org.elasticsearch.search.internal.AliasFilter;
|
||||
import org.elasticsearch.search.internal.InternalSearchResponse;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.elasticsearch.transport.TransportException;
|
||||
|
@ -127,7 +127,7 @@ public class SearchAsyncActionTests extends ESTestCase {
|
|||
new Thread(() -> {
|
||||
Transport.Connection connection = getConnection(null, shard.currentNodeId());
|
||||
TestSearchPhaseResult testSearchPhaseResult = new TestSearchPhaseResult(
|
||||
new SearchContextId(UUIDs.randomBase64UUID(), contextIdGenerator.incrementAndGet()),
|
||||
new ShardSearchContextId(UUIDs.randomBase64UUID(), contextIdGenerator.incrementAndGet()),
|
||||
connection.getNode());
|
||||
listener.onResponse(testSearchPhaseResult);
|
||||
|
||||
|
@ -153,7 +153,7 @@ public class SearchAsyncActionTests extends ESTestCase {
|
|||
asyncAction.start();
|
||||
latch.await();
|
||||
assertTrue(searchPhaseDidRun.get());
|
||||
SearchResponse searchResponse = asyncAction.buildSearchResponse(null, null, asyncAction.buildShardFailures());
|
||||
SearchResponse searchResponse = asyncAction.buildSearchResponse(null, asyncAction.buildShardFailures(), null, null);
|
||||
assertEquals(shardsIter.size() - numSkipped, numRequests.get());
|
||||
assertEquals(0, searchResponse.getFailedShards());
|
||||
assertEquals(numSkipped, searchResponse.getSkippedShards());
|
||||
|
@ -238,7 +238,7 @@ public class SearchAsyncActionTests extends ESTestCase {
|
|||
}
|
||||
Transport.Connection connection = getConnection(null, shard.currentNodeId());
|
||||
TestSearchPhaseResult testSearchPhaseResult = new TestSearchPhaseResult(
|
||||
new SearchContextId(UUIDs.randomBase64UUID(), contextIdGenerator.incrementAndGet()), connection.getNode());
|
||||
new ShardSearchContextId(UUIDs.randomBase64UUID(), contextIdGenerator.incrementAndGet()), connection.getNode());
|
||||
if (shardFailures[shard.shardId().id()]) {
|
||||
listener.onFailure(new RuntimeException());
|
||||
} else {
|
||||
|
@ -282,7 +282,7 @@ public class SearchAsyncActionTests extends ESTestCase {
|
|||
DiscoveryNode primaryNode = new DiscoveryNode("node_1", buildNewFakeTransportAddress(), Version.CURRENT);
|
||||
DiscoveryNode replicaNode = new DiscoveryNode("node_2", buildNewFakeTransportAddress(), Version.CURRENT);
|
||||
|
||||
Map<DiscoveryNode, Set<SearchContextId>> nodeToContextMap = newConcurrentMap();
|
||||
Map<DiscoveryNode, Set<ShardSearchContextId>> nodeToContextMap = newConcurrentMap();
|
||||
AtomicInteger contextIdGenerator = new AtomicInteger(0);
|
||||
int numShards = randomIntBetween(1, 10);
|
||||
GroupShardsIterator<SearchShardIterator> shardsIter = getShardsIter("idx",
|
||||
|
@ -291,7 +291,7 @@ public class SearchAsyncActionTests extends ESTestCase {
|
|||
AtomicInteger numFreedContext = new AtomicInteger();
|
||||
SearchTransportService transportService = new SearchTransportService(null, null) {
|
||||
@Override
|
||||
public void sendFreeContext(Transport.Connection connection, SearchContextId contextId, OriginalIndices originalIndices) {
|
||||
public void sendFreeContext(Transport.Connection connection, ShardSearchContextId contextId, OriginalIndices originalIndices) {
|
||||
numFreedContext.incrementAndGet();
|
||||
assertTrue(nodeToContextMap.containsKey(connection.getNode()));
|
||||
assertTrue(nodeToContextMap.get(connection.getNode()).remove(contextId));
|
||||
|
@ -332,8 +332,8 @@ public class SearchAsyncActionTests extends ESTestCase {
|
|||
assertTrue("shard: " + shard.shardId() + " has been queried twice", response.queried.add(shard.shardId()));
|
||||
Transport.Connection connection = getConnection(null, shard.currentNodeId());
|
||||
TestSearchPhaseResult testSearchPhaseResult = new TestSearchPhaseResult(
|
||||
new SearchContextId(UUIDs.randomBase64UUID(), contextIdGenerator.incrementAndGet()), connection.getNode());
|
||||
Set<SearchContextId> ids = nodeToContextMap.computeIfAbsent(connection.getNode(), (n) -> newConcurrentSet());
|
||||
new ShardSearchContextId(UUIDs.randomBase64UUID(), contextIdGenerator.incrementAndGet()), connection.getNode());
|
||||
Set<ShardSearchContextId> ids = nodeToContextMap.computeIfAbsent(connection.getNode(), (n) -> newConcurrentSet());
|
||||
ids.add(testSearchPhaseResult.getContextId());
|
||||
if (randomBoolean()) {
|
||||
listener.onResponse(testSearchPhaseResult);
|
||||
|
@ -392,7 +392,7 @@ public class SearchAsyncActionTests extends ESTestCase {
|
|||
DiscoveryNode primaryNode = new DiscoveryNode("node_1", buildNewFakeTransportAddress(), Version.CURRENT);
|
||||
DiscoveryNode replicaNode = new DiscoveryNode("node_2", buildNewFakeTransportAddress(), Version.CURRENT);
|
||||
|
||||
Map<DiscoveryNode, Set<SearchContextId>> nodeToContextMap = newConcurrentMap();
|
||||
Map<DiscoveryNode, Set<ShardSearchContextId>> nodeToContextMap = newConcurrentMap();
|
||||
AtomicInteger contextIdGenerator = new AtomicInteger(0);
|
||||
int numShards = randomIntBetween(2, 10);
|
||||
GroupShardsIterator<SearchShardIterator> shardsIter = getShardsIter("idx",
|
||||
|
@ -401,7 +401,7 @@ public class SearchAsyncActionTests extends ESTestCase {
|
|||
AtomicInteger numFreedContext = new AtomicInteger();
|
||||
SearchTransportService transportService = new SearchTransportService(null, null) {
|
||||
@Override
|
||||
public void sendFreeContext(Transport.Connection connection, SearchContextId contextId, OriginalIndices originalIndices) {
|
||||
public void sendFreeContext(Transport.Connection connection, ShardSearchContextId contextId, OriginalIndices originalIndices) {
|
||||
assertNotNull(contextId);
|
||||
numFreedContext.incrementAndGet();
|
||||
assertTrue(nodeToContextMap.containsKey(connection.getNode()));
|
||||
|
@ -446,9 +446,9 @@ public class SearchAsyncActionTests extends ESTestCase {
|
|||
if (shard.shardId().id() == 0) {
|
||||
testSearchPhaseResult = new TestSearchPhaseResult(null, connection.getNode());
|
||||
} else {
|
||||
testSearchPhaseResult = new TestSearchPhaseResult(new SearchContextId(UUIDs.randomBase64UUID(),
|
||||
testSearchPhaseResult = new TestSearchPhaseResult(new ShardSearchContextId(UUIDs.randomBase64UUID(),
|
||||
contextIdGenerator.incrementAndGet()), connection.getNode());
|
||||
Set<SearchContextId> ids = nodeToContextMap.computeIfAbsent(connection.getNode(), (n) -> newConcurrentSet());
|
||||
Set<ShardSearchContextId> ids = nodeToContextMap.computeIfAbsent(connection.getNode(), (n) -> newConcurrentSet());
|
||||
ids.add(testSearchPhaseResult.getContextId());
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
|
@ -547,7 +547,7 @@ public class SearchAsyncActionTests extends ESTestCase {
|
|||
new Thread(() -> {
|
||||
Transport.Connection connection = getConnection(null, shard.currentNodeId());
|
||||
TestSearchPhaseResult testSearchPhaseResult = new TestSearchPhaseResult(
|
||||
new SearchContextId(UUIDs.randomBase64UUID(), contextIdGenerator.incrementAndGet()), connection.getNode());
|
||||
new ShardSearchContextId(UUIDs.randomBase64UUID(), contextIdGenerator.incrementAndGet()), connection.getNode());
|
||||
if (shardIt.remaining() > 0) {
|
||||
numFailReplicas.incrementAndGet();
|
||||
listener.onFailure(new RuntimeException());
|
||||
|
@ -619,13 +619,13 @@ public class SearchAsyncActionTests extends ESTestCase {
|
|||
final Set<ShardId> queried = new HashSet<>();
|
||||
|
||||
TestSearchResponse() {
|
||||
super(InternalSearchResponse.empty(), null, 0, 0, 0, 0L, ShardSearchFailure.EMPTY_ARRAY, Clusters.EMPTY);
|
||||
super(InternalSearchResponse.empty(), null, 0, 0, 0, 0L, ShardSearchFailure.EMPTY_ARRAY, Clusters.EMPTY, null);
|
||||
}
|
||||
}
|
||||
|
||||
public static class TestSearchPhaseResult extends SearchPhaseResult {
|
||||
final DiscoveryNode node;
|
||||
TestSearchPhaseResult(SearchContextId contextId, DiscoveryNode node) {
|
||||
TestSearchPhaseResult(ShardSearchContextId contextId, DiscoveryNode node) {
|
||||
this.contextId = contextId;
|
||||
this.node = node;
|
||||
}
|
||||
|
|
|
@ -0,0 +1,98 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.action.search;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
||||
import org.elasticsearch.common.util.concurrent.AtomicArray;
|
||||
import org.elasticsearch.index.query.IdsQueryBuilder;
|
||||
import org.elasticsearch.index.query.MatchAllQueryBuilder;
|
||||
import org.elasticsearch.index.query.QueryBuilder;
|
||||
import org.elasticsearch.index.query.TermQueryBuilder;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.search.SearchPhaseResult;
|
||||
import org.elasticsearch.search.internal.AliasFilter;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.hasSize;
|
||||
import static org.hamcrest.Matchers.nullValue;
|
||||
|
||||
public class SearchContextIdTests extends ESTestCase {
|
||||
|
||||
QueryBuilder randomQueryBuilder() {
|
||||
if (randomBoolean()) {
|
||||
return new TermQueryBuilder(randomAlphaOfLength(10), randomAlphaOfLength(10));
|
||||
} else if (randomBoolean()) {
|
||||
return new MatchAllQueryBuilder();
|
||||
} else {
|
||||
return new IdsQueryBuilder().addIds(randomAlphaOfLength(10));
|
||||
}
|
||||
}
|
||||
|
||||
public void testEncode() {
|
||||
final NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(Arrays.asList(
|
||||
new NamedWriteableRegistry.Entry(QueryBuilder.class, TermQueryBuilder.NAME, TermQueryBuilder::new),
|
||||
new NamedWriteableRegistry.Entry(QueryBuilder.class, MatchAllQueryBuilder.NAME, MatchAllQueryBuilder::new),
|
||||
new NamedWriteableRegistry.Entry(QueryBuilder.class, IdsQueryBuilder.NAME, IdsQueryBuilder::new)
|
||||
));
|
||||
final AtomicArray<SearchPhaseResult> queryResults = TransportSearchHelperTests.generateQueryResults();
|
||||
final Version version = Version.CURRENT;
|
||||
final Map<String, AliasFilter> aliasFilters = new HashMap<>();
|
||||
for (SearchPhaseResult result : queryResults.asList()) {
|
||||
final AliasFilter aliasFilter;
|
||||
if (randomBoolean()) {
|
||||
aliasFilter = new AliasFilter(randomQueryBuilder());
|
||||
} else if (randomBoolean()) {
|
||||
aliasFilter = new AliasFilter(randomQueryBuilder(), "alias-" + between(1, 10));
|
||||
} else {
|
||||
aliasFilter = AliasFilter.EMPTY;
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
aliasFilters.put(result.getSearchShardTarget().getShardId().getIndex().getUUID(), aliasFilter);
|
||||
}
|
||||
}
|
||||
final String id = SearchContextId.encode(queryResults.asList(), aliasFilters, version);
|
||||
final SearchContextId context = SearchContextId.decode(namedWriteableRegistry, id);
|
||||
assertThat(context.shards().keySet(), hasSize(3));
|
||||
assertThat(context.aliasFilter(), equalTo(aliasFilters));
|
||||
SearchContextIdForNode node1 = context.shards().get(new ShardId("idx", "uuid1", 2));
|
||||
assertThat(node1.getClusterAlias(), equalTo("cluster_x"));
|
||||
assertThat(node1.getNode(), equalTo("node_1"));
|
||||
assertThat(node1.getSearchContextId().getId(), equalTo(1L));
|
||||
assertThat(node1.getSearchContextId().getReaderId(), equalTo("a"));
|
||||
|
||||
SearchContextIdForNode node2 = context.shards().get(new ShardId("idy", "uuid2", 42));
|
||||
assertThat(node2.getClusterAlias(), equalTo("cluster_y"));
|
||||
assertThat(node2.getNode(), equalTo("node_2"));
|
||||
assertThat(node2.getSearchContextId().getId(), equalTo(12L));
|
||||
assertThat(node2.getSearchContextId().getReaderId(), equalTo("b"));
|
||||
|
||||
SearchContextIdForNode node3 = context.shards().get(new ShardId("idy", "uuid2", 43));
|
||||
assertThat(node3.getClusterAlias(), nullValue());
|
||||
assertThat(node3.getNode(), equalTo("node_3"));
|
||||
assertThat(node3.getSearchContextId().getId(), equalTo(42L));
|
||||
assertThat(node3.getSearchContextId().getReaderId(), equalTo("c"));
|
||||
}
|
||||
}
|
|
@ -63,7 +63,7 @@ import org.elasticsearch.search.builder.SearchSourceBuilder;
|
|||
import org.elasticsearch.search.fetch.FetchSearchResult;
|
||||
import org.elasticsearch.search.internal.InternalSearchResponse;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
import org.elasticsearch.search.query.QuerySearchResult;
|
||||
import org.elasticsearch.search.suggest.SortBy;
|
||||
import org.elasticsearch.search.suggest.Suggest;
|
||||
|
@ -295,7 +295,7 @@ public class SearchPhaseControllerTests extends ESTestCase {
|
|||
String clusterAlias = randomBoolean() ? null : "remote";
|
||||
SearchShardTarget searchShardTarget = new SearchShardTarget("", new ShardId("", "", shardIndex),
|
||||
clusterAlias, OriginalIndices.NONE);
|
||||
QuerySearchResult querySearchResult = new QuerySearchResult(new SearchContextId("", shardIndex), searchShardTarget);
|
||||
QuerySearchResult querySearchResult = new QuerySearchResult(new ShardSearchContextId("", shardIndex), searchShardTarget, null);
|
||||
final TopDocs topDocs;
|
||||
float maxScore = 0;
|
||||
if (searchHitsSize == 0) {
|
||||
|
@ -367,7 +367,7 @@ public class SearchPhaseControllerTests extends ESTestCase {
|
|||
float maxScore = -1F;
|
||||
String clusterAlias = randomBoolean() ? null : "remote";
|
||||
SearchShardTarget shardTarget = new SearchShardTarget("", new ShardId("", "", shardIndex), clusterAlias, OriginalIndices.NONE);
|
||||
FetchSearchResult fetchSearchResult = new FetchSearchResult(new SearchContextId("", shardIndex), shardTarget);
|
||||
FetchSearchResult fetchSearchResult = new FetchSearchResult(new ShardSearchContextId("", shardIndex), shardTarget);
|
||||
List<SearchHit> searchHits = new ArrayList<>();
|
||||
for (ScoreDoc scoreDoc : mergedSearchDocs) {
|
||||
if (scoreDoc.shardIndex == shardIndex) {
|
||||
|
@ -432,8 +432,8 @@ public class SearchPhaseControllerTests extends ESTestCase {
|
|||
numEmptyResponses --;
|
||||
}
|
||||
|
||||
QuerySearchResult result = new QuerySearchResult(new SearchContextId(UUIDs.randomBase64UUID(), 0),
|
||||
new SearchShardTarget("node", new ShardId("a", "b", 0), null, OriginalIndices.NONE));
|
||||
QuerySearchResult result = new QuerySearchResult(new ShardSearchContextId("", 0),
|
||||
new SearchShardTarget("node", new ShardId("a", "b", 0), null, OriginalIndices.NONE), null);
|
||||
result.topDocs(new TopDocsAndMaxScore(new TopDocs(new TotalHits(0, TotalHits.Relation.EQUAL_TO), new ScoreDoc[0]), Float.NaN),
|
||||
new DocValueFormat[0]);
|
||||
InternalAggregations aggs = InternalAggregations.from(singletonList(new InternalMax("test", 1.0D, DocValueFormat.RAW, emptyMap())));
|
||||
|
@ -441,8 +441,8 @@ public class SearchPhaseControllerTests extends ESTestCase {
|
|||
result.setShardIndex(0);
|
||||
consumer.consumeResult(result, latch::countDown);
|
||||
|
||||
result = new QuerySearchResult(new SearchContextId(UUIDs.randomBase64UUID(), 1),
|
||||
new SearchShardTarget("node", new ShardId("a", "b", 0), null, OriginalIndices.NONE));
|
||||
result = new QuerySearchResult(new ShardSearchContextId("", 1),
|
||||
new SearchShardTarget("node", new ShardId("a", "b", 0), null, OriginalIndices.NONE), null);
|
||||
result.topDocs(new TopDocsAndMaxScore(new TopDocs(new TotalHits(0, TotalHits.Relation.EQUAL_TO), new ScoreDoc[0]), Float.NaN),
|
||||
new DocValueFormat[0]);
|
||||
aggs = InternalAggregations.from(singletonList(new InternalMax("test", 3.0D, DocValueFormat.RAW, emptyMap())));
|
||||
|
@ -450,8 +450,8 @@ public class SearchPhaseControllerTests extends ESTestCase {
|
|||
result.setShardIndex(2);
|
||||
consumer.consumeResult(result, latch::countDown);
|
||||
|
||||
result = new QuerySearchResult(new SearchContextId(UUIDs.randomBase64UUID(), 1),
|
||||
new SearchShardTarget("node", new ShardId("a", "b", 0), null, OriginalIndices.NONE));
|
||||
result = new QuerySearchResult(new ShardSearchContextId("", 1),
|
||||
new SearchShardTarget("node", new ShardId("a", "b", 0), null, OriginalIndices.NONE), null);
|
||||
result.topDocs(new TopDocsAndMaxScore(new TopDocs(new TotalHits(0, TotalHits.Relation.EQUAL_TO), new ScoreDoc[0]), Float.NaN),
|
||||
new DocValueFormat[0]);
|
||||
aggs = InternalAggregations.from(singletonList(new InternalMax("test", 2.0D, DocValueFormat.RAW, emptyMap())));
|
||||
|
@ -515,8 +515,8 @@ public class SearchPhaseControllerTests extends ESTestCase {
|
|||
threads[i] = new Thread(() -> {
|
||||
int number = randomIntBetween(1, 1000);
|
||||
max.updateAndGet(prev -> Math.max(prev, number));
|
||||
QuerySearchResult result = new QuerySearchResult(new SearchContextId(UUIDs.randomBase64UUID(), id),
|
||||
new SearchShardTarget("node", new ShardId("a", "b", id), null, OriginalIndices.NONE));
|
||||
QuerySearchResult result = new QuerySearchResult(new ShardSearchContextId("", id),
|
||||
new SearchShardTarget("node", new ShardId("a", "b", id), null, OriginalIndices.NONE), null);
|
||||
result.topDocs(new TopDocsAndMaxScore(
|
||||
new TopDocs(new TotalHits(1, TotalHits.Relation.EQUAL_TO), new ScoreDoc[] {new ScoreDoc(0, number)}), number),
|
||||
new DocValueFormat[0]);
|
||||
|
@ -562,8 +562,8 @@ public class SearchPhaseControllerTests extends ESTestCase {
|
|||
for (int i = 0; i < expectedNumResults; i++) {
|
||||
int number = randomIntBetween(1, 1000);
|
||||
max.updateAndGet(prev -> Math.max(prev, number));
|
||||
QuerySearchResult result = new QuerySearchResult(new SearchContextId(UUIDs.randomBase64UUID(), i),
|
||||
new SearchShardTarget("node", new ShardId("a", "b", i), null, OriginalIndices.NONE));
|
||||
QuerySearchResult result = new QuerySearchResult(new ShardSearchContextId("", i),
|
||||
new SearchShardTarget("node", new ShardId("a", "b", i), null, OriginalIndices.NONE), null);
|
||||
result.topDocs(new TopDocsAndMaxScore(new TopDocs(new TotalHits(1, TotalHits.Relation.EQUAL_TO), new ScoreDoc[0]), number),
|
||||
new DocValueFormat[0]);
|
||||
InternalAggregations aggs = InternalAggregations.from(Collections.singletonList(new InternalMax("test", (double) number,
|
||||
|
@ -603,8 +603,8 @@ public class SearchPhaseControllerTests extends ESTestCase {
|
|||
for (int i = 0; i < expectedNumResults; i++) {
|
||||
int number = randomIntBetween(1, 1000);
|
||||
max.updateAndGet(prev -> Math.max(prev, number));
|
||||
QuerySearchResult result = new QuerySearchResult(new SearchContextId(UUIDs.randomBase64UUID(), i),
|
||||
new SearchShardTarget("node", new ShardId("a", "b", i), null, OriginalIndices.NONE));
|
||||
QuerySearchResult result = new QuerySearchResult(new ShardSearchContextId("", i),
|
||||
new SearchShardTarget("node", new ShardId("a", "b", i), null, OriginalIndices.NONE), null);
|
||||
result.topDocs(new TopDocsAndMaxScore(new TopDocs(new TotalHits(1, TotalHits.Relation.EQUAL_TO),
|
||||
new ScoreDoc[] {new ScoreDoc(0, number)}), number), new DocValueFormat[0]);
|
||||
result.setShardIndex(i);
|
||||
|
@ -644,8 +644,8 @@ public class SearchPhaseControllerTests extends ESTestCase {
|
|||
int score = 100;
|
||||
CountDownLatch latch = new CountDownLatch(4);
|
||||
for (int i = 0; i < 4; i++) {
|
||||
QuerySearchResult result = new QuerySearchResult(new SearchContextId(UUIDs.randomBase64UUID(), i),
|
||||
new SearchShardTarget("node", new ShardId("a", "b", i), null, OriginalIndices.NONE));
|
||||
QuerySearchResult result = new QuerySearchResult(new ShardSearchContextId("", i),
|
||||
new SearchShardTarget("node", new ShardId("a", "b", i), null, OriginalIndices.NONE), null);
|
||||
ScoreDoc[] docs = new ScoreDoc[3];
|
||||
for (int j = 0; j < docs.length; j++) {
|
||||
docs[j] = new ScoreDoc(0, score--);
|
||||
|
@ -688,8 +688,8 @@ public class SearchPhaseControllerTests extends ESTestCase {
|
|||
max.updateAndGet(prev -> Math.max(prev, number));
|
||||
FieldDoc[] fieldDocs = {new FieldDoc(0, Float.NaN, new Object[]{number})};
|
||||
TopDocs topDocs = new TopFieldDocs(new TotalHits(1, Relation.EQUAL_TO), fieldDocs, sortFields);
|
||||
QuerySearchResult result = new QuerySearchResult(new SearchContextId(UUIDs.randomBase64UUID(), i),
|
||||
new SearchShardTarget("node", new ShardId("a", "b", i), null, OriginalIndices.NONE));
|
||||
QuerySearchResult result = new QuerySearchResult(new ShardSearchContextId("", i),
|
||||
new SearchShardTarget("node", new ShardId("a", "b", i), null, OriginalIndices.NONE), null);
|
||||
result.topDocs(new TopDocsAndMaxScore(topDocs, Float.NaN), docValueFormats);
|
||||
result.setShardIndex(i);
|
||||
result.size(size);
|
||||
|
@ -728,8 +728,8 @@ public class SearchPhaseControllerTests extends ESTestCase {
|
|||
Object[] values = {randomFrom(collapseValues)};
|
||||
FieldDoc[] fieldDocs = {new FieldDoc(0, Float.NaN, values)};
|
||||
TopDocs topDocs = new CollapseTopFieldDocs("field", new TotalHits(1, Relation.EQUAL_TO), fieldDocs, sortFields, values);
|
||||
QuerySearchResult result = new QuerySearchResult(new SearchContextId(UUIDs.randomBase64UUID(), i),
|
||||
new SearchShardTarget("node", new ShardId("a", "b", i), null, OriginalIndices.NONE));
|
||||
QuerySearchResult result = new QuerySearchResult(new ShardSearchContextId("", i),
|
||||
new SearchShardTarget("node", new ShardId("a", "b", i), null, OriginalIndices.NONE), null);
|
||||
result.topDocs(new TopDocsAndMaxScore(topDocs, Float.NaN), docValueFormats);
|
||||
result.setShardIndex(i);
|
||||
result.size(size);
|
||||
|
@ -763,8 +763,8 @@ public class SearchPhaseControllerTests extends ESTestCase {
|
|||
int maxScoreCompletion = -1;
|
||||
CountDownLatch latch = new CountDownLatch(expectedNumResults);
|
||||
for (int i = 0; i < expectedNumResults; i++) {
|
||||
QuerySearchResult result = new QuerySearchResult(new SearchContextId(UUIDs.randomBase64UUID(), i),
|
||||
new SearchShardTarget("node", new ShardId("a", "b", i), null, OriginalIndices.NONE));
|
||||
QuerySearchResult result = new QuerySearchResult(new ShardSearchContextId("", i),
|
||||
new SearchShardTarget("node", new ShardId("a", "b", i), null, OriginalIndices.NONE), null);
|
||||
List<Suggest.Suggestion<? extends Suggest.Suggestion.Entry<? extends Suggest.Suggestion.Entry.Option>>> suggestions =
|
||||
new ArrayList<>();
|
||||
{
|
||||
|
@ -892,8 +892,8 @@ public class SearchPhaseControllerTests extends ESTestCase {
|
|||
threads[i] = new Thread(() -> {
|
||||
int number = randomIntBetween(1, 1000);
|
||||
max.updateAndGet(prev -> Math.max(prev, number));
|
||||
QuerySearchResult result = new QuerySearchResult(new SearchContextId(UUIDs.randomBase64UUID(), id),
|
||||
new SearchShardTarget("node", new ShardId("a", "b", id), null, OriginalIndices.NONE));
|
||||
QuerySearchResult result = new QuerySearchResult(new ShardSearchContextId("", id),
|
||||
new SearchShardTarget("node", new ShardId("a", "b", id), null, OriginalIndices.NONE), null);
|
||||
result.topDocs(new TopDocsAndMaxScore(
|
||||
new TopDocs(new TotalHits(1, TotalHits.Relation.EQUAL_TO), new ScoreDoc[]{new ScoreDoc(0, number)}), number),
|
||||
new DocValueFormat[0]);
|
||||
|
@ -948,8 +948,9 @@ public class SearchPhaseControllerTests extends ESTestCase {
|
|||
for (int i = 0; i < expectedNumResults; i++) {
|
||||
final int index = i;
|
||||
threads[index] = new Thread(() -> {
|
||||
QuerySearchResult result = new QuerySearchResult(new SearchContextId(UUIDs.randomBase64UUID(), index),
|
||||
new SearchShardTarget("node", new ShardId("a", "b", index), null, OriginalIndices.NONE));
|
||||
QuerySearchResult result = new QuerySearchResult(new ShardSearchContextId(UUIDs.randomBase64UUID(), index),
|
||||
new SearchShardTarget("node", new ShardId("a", "b", index), null, OriginalIndices.NONE),
|
||||
null);
|
||||
result.topDocs(new TopDocsAndMaxScore(
|
||||
new TopDocs(new TotalHits(0, TotalHits.Relation.EQUAL_TO), Lucene.EMPTY_SCORE_DOCS), Float.NaN),
|
||||
new DocValueFormat[0]);
|
||||
|
|
|
@ -39,7 +39,7 @@ import org.elasticsearch.search.SearchShardTarget;
|
|||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||
import org.elasticsearch.search.collapse.CollapseBuilder;
|
||||
import org.elasticsearch.search.internal.AliasFilter;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchRequest;
|
||||
import org.elasticsearch.search.query.QuerySearchResult;
|
||||
import org.elasticsearch.search.sort.SortBuilders;
|
||||
|
@ -98,8 +98,8 @@ public class SearchQueryThenFetchAsyncActionTests extends ESTestCase {
|
|||
assertNotEquals(shardId, (int) request.getBottomSortValues().getFormattedSortValues()[0]);
|
||||
numWithTopDocs.incrementAndGet();
|
||||
}
|
||||
QuerySearchResult queryResult = new QuerySearchResult(new SearchContextId("N/A", 123),
|
||||
new SearchShardTarget("node1", new ShardId("idx", "na", shardId), null, OriginalIndices.NONE));
|
||||
QuerySearchResult queryResult = new QuerySearchResult(new ShardSearchContextId("N/A", 123),
|
||||
new SearchShardTarget("node1", new ShardId("idx", "na", shardId), null, OriginalIndices.NONE), null);
|
||||
SortField sortField = new SortField("timestamp", SortField.Type.LONG);
|
||||
if (withCollapse) {
|
||||
queryResult.topDocs(new TopDocsAndMaxScore(
|
||||
|
|
|
@ -205,6 +205,39 @@ public class SearchRequestTests extends AbstractSearchTestCase {
|
|||
assertEquals(1, validationErrors.validationErrors().size());
|
||||
assertEquals("using [rescore] is not allowed in a scroll context", validationErrors.validationErrors().get(0));
|
||||
}
|
||||
{
|
||||
// Reader context with scroll
|
||||
SearchRequest searchRequest = new SearchRequest()
|
||||
.source(new SearchSourceBuilder().pointInTimeBuilder(
|
||||
new SearchSourceBuilder.PointInTimeBuilder("id", TimeValue.timeValueMillis(randomIntBetween(1, 10)))))
|
||||
.scroll(TimeValue.timeValueMillis(randomIntBetween(1, 100)));
|
||||
ActionRequestValidationException validationErrors = searchRequest.validate();
|
||||
assertNotNull(validationErrors);
|
||||
assertEquals(1, validationErrors.validationErrors().size());
|
||||
assertEquals("using [point in time] is not allowed in a scroll context", validationErrors.validationErrors().get(0));
|
||||
}
|
||||
{
|
||||
// Reader context with preference
|
||||
SearchRequest searchRequest = new SearchRequest()
|
||||
.source(new SearchSourceBuilder().
|
||||
pointInTimeBuilder(new SearchSourceBuilder.PointInTimeBuilder("id", TimeValue.timeValueMillis(between(1, 10)))))
|
||||
.preference("test");
|
||||
ActionRequestValidationException validationErrors = searchRequest.validate();
|
||||
assertNotNull(validationErrors);
|
||||
assertEquals(1, validationErrors.validationErrors().size());
|
||||
assertEquals("[preference] cannot be used with point in time", validationErrors.validationErrors().get(0));
|
||||
}
|
||||
{
|
||||
// Reader context with routing
|
||||
SearchRequest searchRequest = new SearchRequest()
|
||||
.source(new SearchSourceBuilder()
|
||||
.pointInTimeBuilder(new SearchSourceBuilder.PointInTimeBuilder("id", TimeValue.timeValueMillis(between(1, 10)))))
|
||||
.routing("test");
|
||||
ActionRequestValidationException validationErrors = searchRequest.validate();
|
||||
assertNotNull(validationErrors);
|
||||
assertEquals(1, validationErrors.validationErrors().size());
|
||||
assertEquals("[routing] cannot be used with point in time", validationErrors.validationErrors().get(0));
|
||||
}
|
||||
}
|
||||
|
||||
public void testCopyConstructor() throws IOException {
|
||||
|
|
|
@ -30,7 +30,7 @@ import org.elasticsearch.index.shard.ShardId;
|
|||
import org.elasticsearch.search.Scroll;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.internal.InternalScrollSearchRequest;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
|
||||
|
@ -47,11 +47,11 @@ public class SearchScrollAsyncActionTests extends ESTestCase {
|
|||
public void testSendRequestsToNodes() throws InterruptedException {
|
||||
|
||||
ParsedScrollId scrollId = getParsedScrollId(
|
||||
new ScrollIdForNode(null, "node1", new SearchContextId(UUIDs.randomBase64UUID(), 1)),
|
||||
new ScrollIdForNode(null, "node2", new SearchContextId(UUIDs.randomBase64UUID(), 2)),
|
||||
new ScrollIdForNode(null, "node3", new SearchContextId(UUIDs.randomBase64UUID(), 17)),
|
||||
new ScrollIdForNode(null, "node1", new SearchContextId(UUIDs.randomBase64UUID(), 0)),
|
||||
new ScrollIdForNode(null, "node3", new SearchContextId(UUIDs.randomBase64UUID(), 0)));
|
||||
new SearchContextIdForNode(null, "node1", new ShardSearchContextId(UUIDs.randomBase64UUID(), 1)),
|
||||
new SearchContextIdForNode(null, "node2", new ShardSearchContextId(UUIDs.randomBase64UUID(), 2)),
|
||||
new SearchContextIdForNode(null, "node3", new ShardSearchContextId(UUIDs.randomBase64UUID(), 17)),
|
||||
new SearchContextIdForNode(null, "node1", new ShardSearchContextId(UUIDs.randomBase64UUID(), 0)),
|
||||
new SearchContextIdForNode(null, "node3", new ShardSearchContextId(UUIDs.randomBase64UUID(), 0)));
|
||||
DiscoveryNodes discoveryNodes = DiscoveryNodes.builder()
|
||||
.add(new DiscoveryNode("node1", buildNewFakeTransportAddress(), Version.CURRENT))
|
||||
.add(new DiscoveryNode("node2", buildNewFakeTransportAddress(), Version.CURRENT))
|
||||
|
@ -105,10 +105,10 @@ public class SearchScrollAsyncActionTests extends ESTestCase {
|
|||
latch.await();
|
||||
ShardSearchFailure[] shardSearchFailures = action.buildShardFailures();
|
||||
assertEquals(0, shardSearchFailures.length);
|
||||
ScrollIdForNode[] context = scrollId.getContext();
|
||||
SearchContextIdForNode[] context = scrollId.getContext();
|
||||
for (int i = 0; i < results.length(); i++) {
|
||||
assertNotNull(results.get(i));
|
||||
assertEquals(context[i].getContextId(), results.get(i).getContextId());
|
||||
assertEquals(context[i].getSearchContextId(), results.get(i).getContextId());
|
||||
assertEquals(context[i].getNode(), results.get(i).node.getId());
|
||||
}
|
||||
}
|
||||
|
@ -116,11 +116,11 @@ public class SearchScrollAsyncActionTests extends ESTestCase {
|
|||
public void testFailNextPhase() throws InterruptedException {
|
||||
|
||||
ParsedScrollId scrollId = getParsedScrollId(
|
||||
new ScrollIdForNode(null, "node1", new SearchContextId("", 1)),
|
||||
new ScrollIdForNode(null, "node2", new SearchContextId("a", 2)),
|
||||
new ScrollIdForNode(null, "node3", new SearchContextId("b", 17)),
|
||||
new ScrollIdForNode(null, "node1", new SearchContextId("c", 0)),
|
||||
new ScrollIdForNode(null, "node3", new SearchContextId("d", 0)));
|
||||
new SearchContextIdForNode(null, "node1", new ShardSearchContextId("", 1)),
|
||||
new SearchContextIdForNode(null, "node2", new ShardSearchContextId("a", 2)),
|
||||
new SearchContextIdForNode(null, "node3", new ShardSearchContextId("b", 17)),
|
||||
new SearchContextIdForNode(null, "node1", new ShardSearchContextId("c", 0)),
|
||||
new SearchContextIdForNode(null, "node3", new ShardSearchContextId("d", 0)));
|
||||
DiscoveryNodes discoveryNodes = DiscoveryNodes.builder()
|
||||
.add(new DiscoveryNode("node1", buildNewFakeTransportAddress(), Version.CURRENT))
|
||||
.add(new DiscoveryNode("node2", buildNewFakeTransportAddress(), Version.CURRENT))
|
||||
|
@ -196,21 +196,21 @@ public class SearchScrollAsyncActionTests extends ESTestCase {
|
|||
latch.await();
|
||||
ShardSearchFailure[] shardSearchFailures = action.buildShardFailures();
|
||||
assertEquals(0, shardSearchFailures.length);
|
||||
ScrollIdForNode[] context = scrollId.getContext();
|
||||
SearchContextIdForNode[] context = scrollId.getContext();
|
||||
for (int i = 0; i < results.length(); i++) {
|
||||
assertNotNull(results.get(i));
|
||||
assertEquals(context[i].getContextId(), results.get(i).getContextId());
|
||||
assertEquals(context[i].getSearchContextId(), results.get(i).getContextId());
|
||||
assertEquals(context[i].getNode(), results.get(i).node.getId());
|
||||
}
|
||||
}
|
||||
|
||||
public void testNodeNotAvailable() throws InterruptedException {
|
||||
ParsedScrollId scrollId = getParsedScrollId(
|
||||
new ScrollIdForNode(null, "node1", new SearchContextId("", 1)),
|
||||
new ScrollIdForNode(null, "node2", new SearchContextId("", 2)),
|
||||
new ScrollIdForNode(null, "node3", new SearchContextId("", 17)),
|
||||
new ScrollIdForNode(null, "node1", new SearchContextId("", 0)),
|
||||
new ScrollIdForNode(null, "node3", new SearchContextId("", 0)));
|
||||
new SearchContextIdForNode(null, "node1", new ShardSearchContextId("", 1)),
|
||||
new SearchContextIdForNode(null, "node2", new ShardSearchContextId("", 2)),
|
||||
new SearchContextIdForNode(null, "node3", new ShardSearchContextId("", 17)),
|
||||
new SearchContextIdForNode(null, "node1", new ShardSearchContextId("", 0)),
|
||||
new SearchContextIdForNode(null, "node3", new ShardSearchContextId("", 0)));
|
||||
// node2 is not available
|
||||
DiscoveryNodes discoveryNodes = DiscoveryNodes.builder()
|
||||
.add(new DiscoveryNode("node1", buildNewFakeTransportAddress(), Version.CURRENT))
|
||||
|
@ -271,13 +271,13 @@ public class SearchScrollAsyncActionTests extends ESTestCase {
|
|||
assertEquals(1, shardSearchFailures.length);
|
||||
assertEquals("IllegalStateException[node [node2] is not available]", shardSearchFailures[0].reason());
|
||||
|
||||
ScrollIdForNode[] context = scrollId.getContext();
|
||||
SearchContextIdForNode[] context = scrollId.getContext();
|
||||
for (int i = 0; i < results.length(); i++) {
|
||||
if (context[i].getNode().equals("node2")) {
|
||||
assertNull(results.get(i));
|
||||
} else {
|
||||
assertNotNull(results.get(i));
|
||||
assertEquals(context[i].getContextId(), results.get(i).getContextId());
|
||||
assertEquals(context[i].getSearchContextId(), results.get(i).getContextId());
|
||||
assertEquals(context[i].getNode(), results.get(i).node.getId());
|
||||
}
|
||||
}
|
||||
|
@ -285,11 +285,11 @@ public class SearchScrollAsyncActionTests extends ESTestCase {
|
|||
|
||||
public void testShardFailures() throws InterruptedException {
|
||||
ParsedScrollId scrollId = getParsedScrollId(
|
||||
new ScrollIdForNode(null, "node1", new SearchContextId("", 1)),
|
||||
new ScrollIdForNode(null, "node2", new SearchContextId("", 2)),
|
||||
new ScrollIdForNode(null, "node3", new SearchContextId("",17)),
|
||||
new ScrollIdForNode(null, "node1", new SearchContextId("", 0)),
|
||||
new ScrollIdForNode(null, "node3", new SearchContextId("", 0)));
|
||||
new SearchContextIdForNode(null, "node1", new ShardSearchContextId("", 1)),
|
||||
new SearchContextIdForNode(null, "node2", new ShardSearchContextId("", 2)),
|
||||
new SearchContextIdForNode(null, "node3", new ShardSearchContextId("",17)),
|
||||
new SearchContextIdForNode(null, "node1", new ShardSearchContextId("", 0)),
|
||||
new SearchContextIdForNode(null, "node3", new ShardSearchContextId("", 0)));
|
||||
DiscoveryNodes discoveryNodes = DiscoveryNodes.builder()
|
||||
.add(new DiscoveryNode("node1", buildNewFakeTransportAddress(), Version.CURRENT))
|
||||
.add(new DiscoveryNode("node2", buildNewFakeTransportAddress(), Version.CURRENT))
|
||||
|
@ -349,13 +349,13 @@ public class SearchScrollAsyncActionTests extends ESTestCase {
|
|||
assertEquals(1, shardSearchFailures.length);
|
||||
assertEquals("IllegalArgumentException[BOOM on shard]", shardSearchFailures[0].reason());
|
||||
|
||||
ScrollIdForNode[] context = scrollId.getContext();
|
||||
SearchContextIdForNode[] context = scrollId.getContext();
|
||||
for (int i = 0; i < results.length(); i++) {
|
||||
if (context[i].getContextId().getId() == 17) {
|
||||
if (context[i].getSearchContextId().getId() == 17) {
|
||||
assertNull(results.get(i));
|
||||
} else {
|
||||
assertNotNull(results.get(i));
|
||||
assertEquals(context[i].getContextId(), results.get(i).getContextId());
|
||||
assertEquals(context[i].getSearchContextId(), results.get(i).getContextId());
|
||||
assertEquals(context[i].getNode(), results.get(i).node.getId());
|
||||
}
|
||||
}
|
||||
|
@ -363,11 +363,11 @@ public class SearchScrollAsyncActionTests extends ESTestCase {
|
|||
|
||||
public void testAllShardsFailed() throws InterruptedException {
|
||||
ParsedScrollId scrollId = getParsedScrollId(
|
||||
new ScrollIdForNode(null, "node1", new SearchContextId("", 1)),
|
||||
new ScrollIdForNode(null, "node2", new SearchContextId("", 2)),
|
||||
new ScrollIdForNode(null, "node3", new SearchContextId("", 17)),
|
||||
new ScrollIdForNode(null, "node1", new SearchContextId("", 0)),
|
||||
new ScrollIdForNode(null, "node3", new SearchContextId("", 0)));
|
||||
new SearchContextIdForNode(null, "node1", new ShardSearchContextId("", 1)),
|
||||
new SearchContextIdForNode(null, "node2", new ShardSearchContextId("", 2)),
|
||||
new SearchContextIdForNode(null, "node3", new ShardSearchContextId("", 17)),
|
||||
new SearchContextIdForNode(null, "node1", new ShardSearchContextId("", 0)),
|
||||
new SearchContextIdForNode(null, "node3", new ShardSearchContextId("", 0)));
|
||||
DiscoveryNodes discoveryNodes = DiscoveryNodes.builder()
|
||||
.add(new DiscoveryNode("node1", buildNewFakeTransportAddress(), Version.CURRENT))
|
||||
.add(new DiscoveryNode("node2", buildNewFakeTransportAddress(), Version.CURRENT))
|
||||
|
@ -429,7 +429,7 @@ public class SearchScrollAsyncActionTests extends ESTestCase {
|
|||
|
||||
action.run();
|
||||
latch.await();
|
||||
ScrollIdForNode[] context = scrollId.getContext();
|
||||
SearchContextIdForNode[] context = scrollId.getContext();
|
||||
|
||||
ShardSearchFailure[] shardSearchFailures = action.buildShardFailures();
|
||||
assertEquals(context.length, shardSearchFailures.length);
|
||||
|
@ -440,10 +440,10 @@ public class SearchScrollAsyncActionTests extends ESTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
private static ParsedScrollId getParsedScrollId(ScrollIdForNode... idsForNodes) {
|
||||
List<ScrollIdForNode> scrollIdForNodes = Arrays.asList(idsForNodes);
|
||||
Collections.shuffle(scrollIdForNodes, random());
|
||||
return new ParsedScrollId("", "test", scrollIdForNodes.toArray(new ScrollIdForNode[0]));
|
||||
private static ParsedScrollId getParsedScrollId(SearchContextIdForNode... idsForNodes) {
|
||||
List<SearchContextIdForNode> searchContextIdForNodes = Arrays.asList(idsForNodes);
|
||||
Collections.shuffle(searchContextIdForNodes, random());
|
||||
return new ParsedScrollId("", "test", searchContextIdForNodes.toArray(new SearchContextIdForNode[0]));
|
||||
}
|
||||
|
||||
private ActionListener<SearchResponse> dummyListener() {
|
||||
|
|
|
@ -33,7 +33,7 @@ import org.elasticsearch.common.xcontent.XContentParser;
|
|||
import org.elasticsearch.common.xcontent.XContentType;
|
||||
import org.elasticsearch.common.xcontent.json.JsonXContent;
|
||||
import org.elasticsearch.search.internal.InternalScrollSearchRequest;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -60,7 +60,7 @@ public class SearchScrollRequestTests extends ESTestCase {
|
|||
public void testInternalScrollSearchRequestSerialization() throws IOException {
|
||||
SearchScrollRequest searchScrollRequest = createSearchScrollRequest();
|
||||
InternalScrollSearchRequest internalScrollSearchRequest =
|
||||
new InternalScrollSearchRequest(searchScrollRequest, new SearchContextId(UUIDs.randomBase64UUID(), randomLong()));
|
||||
new InternalScrollSearchRequest(searchScrollRequest, new ShardSearchContextId(UUIDs.randomBase64UUID(), randomLong()));
|
||||
try (BytesStreamOutput output = new BytesStreamOutput()) {
|
||||
internalScrollSearchRequest.writeTo(output);
|
||||
try (StreamInput in = output.bytes().streamInput()) {
|
||||
|
|
|
@ -36,8 +36,6 @@ import org.elasticsearch.cluster.metadata.IndexMetadata;
|
|||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.routing.GroupShardsIterator;
|
||||
import org.elasticsearch.cluster.routing.GroupShardsIteratorTests;
|
||||
import org.elasticsearch.cluster.routing.PlainShardIterator;
|
||||
import org.elasticsearch.cluster.routing.ShardIterator;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.cluster.routing.ShardRoutingState;
|
||||
import org.elasticsearch.cluster.routing.TestShardRouting;
|
||||
|
@ -144,7 +142,7 @@ public class TransportSearchActionTests extends ESTestCase {
|
|||
List<SearchShardIterator> expected = new ArrayList<>();
|
||||
String localClusterAlias = randomAlphaOfLengthBetween(5, 10);
|
||||
OriginalIndices localIndices = OriginalIndicesTests.randomOriginalIndices();
|
||||
List<ShardIterator> localShardIterators = new ArrayList<>();
|
||||
List<SearchShardIterator> localShardIterators = new ArrayList<>();
|
||||
List<SearchShardIterator> remoteShardIterators = new ArrayList<>();
|
||||
int numShards = randomIntBetween(0, 10);
|
||||
for (int i = 0; i < numShards; i++) {
|
||||
|
@ -154,7 +152,7 @@ public class TransportSearchActionTests extends ESTestCase {
|
|||
boolean localIndex = randomBoolean();
|
||||
if (localIndex) {
|
||||
SearchShardIterator localIterator = createSearchShardIterator(i, index, localIndices, localClusterAlias);
|
||||
localShardIterators.add(new PlainShardIterator(localIterator.shardId(), localIterator.getShardRoutings()));
|
||||
localShardIterators.add(localIterator);
|
||||
if (rarely()) {
|
||||
String remoteClusterAlias = randomFrom(remoteClusters);
|
||||
//simulate scenario where the local cluster is also registered as a remote one
|
||||
|
@ -191,11 +189,12 @@ public class TransportSearchActionTests extends ESTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
Collections.shuffle(localShardIterators, random());
|
||||
Collections.shuffle(remoteShardIterators, random());
|
||||
|
||||
GroupShardsIterator<SearchShardIterator> groupShardsIterator = TransportSearchAction.mergeShardsIterators(
|
||||
new GroupShardsIterator<>(localShardIterators), localIndices, localClusterAlias, remoteShardIterators);
|
||||
GroupShardsIterator<SearchShardIterator> groupShardsIterator =
|
||||
TransportSearchAction.mergeShardsIterators(localShardIterators, remoteShardIterators);
|
||||
List<SearchShardIterator> result = new ArrayList<>();
|
||||
for (SearchShardIterator searchShardIterator : groupShardsIterator) {
|
||||
result.add(searchShardIterator);
|
||||
|
@ -367,7 +366,7 @@ public class TransportSearchActionTests extends ESTestCase {
|
|||
private static SearchResponse emptySearchResponse() {
|
||||
InternalSearchResponse response = new InternalSearchResponse(new SearchHits(new SearchHit[0],
|
||||
new TotalHits(0, TotalHits.Relation.EQUAL_TO), Float.NaN), InternalAggregations.EMPTY, null, null, false, null, 1);
|
||||
return new SearchResponse(response, null, 1, 1, 0, 100, ShardSearchFailure.EMPTY_ARRAY, SearchResponse.Clusters.EMPTY);
|
||||
return new SearchResponse(response, null, 1, 1, 0, 100, ShardSearchFailure.EMPTY_ARRAY, SearchResponse.Clusters.EMPTY, null);
|
||||
}
|
||||
|
||||
public void testCCSRemoteReduceMergeFails() throws Exception {
|
||||
|
@ -846,10 +845,9 @@ public class TransportSearchActionTests extends ESTestCase {
|
|||
|
||||
public void testShouldPreFilterSearchShards() {
|
||||
int numIndices = randomIntBetween(2, 10);
|
||||
Index[] indices = new Index[numIndices];
|
||||
String[] indices = new String[numIndices];
|
||||
for (int i = 0; i < numIndices; i++) {
|
||||
String indexName = randomAlphaOfLengthBetween(5, 10);
|
||||
indices[i] = new Index(indexName, indexName + "-uuid");
|
||||
indices[i] = randomAlphaOfLengthBetween(5, 10);
|
||||
}
|
||||
ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT).build();
|
||||
{
|
||||
|
@ -889,16 +887,15 @@ public class TransportSearchActionTests extends ESTestCase {
|
|||
public void testShouldPreFilterSearchShardsWithReadOnly() {
|
||||
int numIndices = randomIntBetween(2, 10);
|
||||
int numReadOnly = randomIntBetween(1, numIndices);
|
||||
Index[] indices = new Index[numIndices];
|
||||
String[] indices = new String[numIndices];
|
||||
ClusterBlocks.Builder blocksBuilder = ClusterBlocks.builder();
|
||||
for (int i = 0; i < numIndices; i++) {
|
||||
String indexName = randomAlphaOfLengthBetween(5, 10);
|
||||
indices[i] = new Index(indexName, indexName + "-uuid");
|
||||
indices[i] = randomAlphaOfLengthBetween(5, 10);;
|
||||
if (--numReadOnly >= 0) {
|
||||
if (randomBoolean()) {
|
||||
blocksBuilder.addIndexBlock(indexName, IndexMetadata.INDEX_WRITE_BLOCK);
|
||||
blocksBuilder.addIndexBlock(indices[i], IndexMetadata.INDEX_WRITE_BLOCK);
|
||||
} else {
|
||||
blocksBuilder.addIndexBlock(indexName, IndexMetadata.INDEX_READ_ONLY_BLOCK);
|
||||
blocksBuilder.addIndexBlock(indices[i], IndexMetadata.INDEX_READ_ONLY_BLOCK);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,62 +24,66 @@ import org.elasticsearch.common.util.concurrent.AtomicArray;
|
|||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.search.SearchPhaseResult;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.elasticsearch.test.VersionUtils;
|
||||
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
|
||||
public class TransportSearchHelperTests extends ESTestCase {
|
||||
|
||||
public void testParseScrollId() throws IOException {
|
||||
public static AtomicArray<SearchPhaseResult> generateQueryResults() {
|
||||
AtomicArray<SearchPhaseResult> array = new AtomicArray<>(3);
|
||||
DiscoveryNode node1 = new DiscoveryNode("node_1", buildNewFakeTransportAddress(), Version.CURRENT);
|
||||
DiscoveryNode node2 = new DiscoveryNode("node_2", buildNewFakeTransportAddress(), Version.CURRENT);
|
||||
DiscoveryNode node3 = new DiscoveryNode("node_3", buildNewFakeTransportAddress(), Version.CURRENT);
|
||||
SearchAsyncActionTests.TestSearchPhaseResult testSearchPhaseResult1 =
|
||||
new SearchAsyncActionTests.TestSearchPhaseResult(new SearchContextId("x", 1), node1);
|
||||
new SearchAsyncActionTests.TestSearchPhaseResult(new ShardSearchContextId("a", 1), node1);
|
||||
testSearchPhaseResult1.setSearchShardTarget(new SearchShardTarget("node_1", new ShardId("idx", "uuid1", 2), "cluster_x", null));
|
||||
SearchAsyncActionTests.TestSearchPhaseResult testSearchPhaseResult2 =
|
||||
new SearchAsyncActionTests.TestSearchPhaseResult(new SearchContextId("y", 12), node2);
|
||||
new SearchAsyncActionTests.TestSearchPhaseResult(new ShardSearchContextId("b", 12), node2);
|
||||
testSearchPhaseResult2.setSearchShardTarget(new SearchShardTarget("node_2", new ShardId("idy", "uuid2", 42), "cluster_y", null));
|
||||
SearchAsyncActionTests.TestSearchPhaseResult testSearchPhaseResult3 =
|
||||
new SearchAsyncActionTests.TestSearchPhaseResult(new SearchContextId("z", 42), node3);
|
||||
new SearchAsyncActionTests.TestSearchPhaseResult(new ShardSearchContextId("c", 42), node3);
|
||||
testSearchPhaseResult3.setSearchShardTarget(new SearchShardTarget("node_3", new ShardId("idy", "uuid2", 43), null, null));
|
||||
array.setOnce(0, testSearchPhaseResult1);
|
||||
array.setOnce(1, testSearchPhaseResult2);
|
||||
array.setOnce(2, testSearchPhaseResult3);
|
||||
return array;
|
||||
}
|
||||
|
||||
boolean includeUUID = randomBoolean();
|
||||
String scrollId = TransportSearchHelper.buildScrollId(array, includeUUID);
|
||||
public void testParseScrollId() {
|
||||
final Version version = VersionUtils.randomVersion(random());
|
||||
boolean includeUUID = version.onOrAfter(Version.V_7_7_0);
|
||||
final AtomicArray<SearchPhaseResult> queryResults = generateQueryResults();
|
||||
String scrollId = TransportSearchHelper.buildScrollId(queryResults, version);
|
||||
ParsedScrollId parseScrollId = TransportSearchHelper.parseScrollId(scrollId);
|
||||
assertEquals(3, parseScrollId.getContext().length);
|
||||
assertEquals("node_1", parseScrollId.getContext()[0].getNode());
|
||||
assertEquals("cluster_x", parseScrollId.getContext()[0].getClusterAlias());
|
||||
assertEquals(1, parseScrollId.getContext()[0].getContextId().getId());
|
||||
assertEquals(1, parseScrollId.getContext()[0].getSearchContextId().getId());
|
||||
if (includeUUID) {
|
||||
assertThat(parseScrollId.getContext()[0].getContextId().getReaderId(), equalTo("x"));
|
||||
assertThat(parseScrollId.getContext()[0].getSearchContextId().getReaderId(), equalTo("a"));
|
||||
} else {
|
||||
assertThat(parseScrollId.getContext()[0].getContextId().getReaderId(), equalTo(""));
|
||||
assertThat(parseScrollId.getContext()[0].getSearchContextId().getReaderId(), equalTo(""));
|
||||
}
|
||||
|
||||
assertEquals("node_2", parseScrollId.getContext()[1].getNode());
|
||||
assertEquals("cluster_y", parseScrollId.getContext()[1].getClusterAlias());
|
||||
assertEquals(12, parseScrollId.getContext()[1].getContextId().getId());
|
||||
assertEquals(12, parseScrollId.getContext()[1].getSearchContextId().getId());
|
||||
if (includeUUID) {
|
||||
assertThat(parseScrollId.getContext()[1].getContextId().getReaderId(), equalTo("y"));
|
||||
assertThat(parseScrollId.getContext()[1].getSearchContextId().getReaderId(), equalTo("b"));
|
||||
} else {
|
||||
assertThat(parseScrollId.getContext()[1].getContextId().getReaderId(), equalTo(""));
|
||||
assertThat(parseScrollId.getContext()[1].getSearchContextId().getReaderId(), equalTo(""));
|
||||
}
|
||||
|
||||
assertEquals("node_3", parseScrollId.getContext()[2].getNode());
|
||||
assertNull(parseScrollId.getContext()[2].getClusterAlias());
|
||||
assertEquals(42, parseScrollId.getContext()[2].getContextId().getId());
|
||||
assertEquals(42, parseScrollId.getContext()[2].getSearchContextId().getId());
|
||||
if (includeUUID) {
|
||||
assertThat(parseScrollId.getContext()[2].getContextId().getReaderId(), equalTo("z"));
|
||||
assertThat(parseScrollId.getContext()[2].getSearchContextId().getReaderId(), equalTo("c"));
|
||||
} else {
|
||||
assertThat(parseScrollId.getContext()[2].getContextId().getReaderId(), equalTo(""));
|
||||
assertThat(parseScrollId.getContext()[2].getSearchContextId().getReaderId(), equalTo(""));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.elasticsearch.action.support.ActionFilters;
|
|||
import org.elasticsearch.action.support.TransportAction;
|
||||
import org.elasticsearch.client.AbstractClientHeadersTestCase;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.tasks.TaskManager;
|
||||
|
@ -43,7 +44,8 @@ public class NodeClientHeadersTests extends AbstractClientHeadersTestCase {
|
|||
Settings settings = HEADER_SETTINGS;
|
||||
Actions actions = new Actions(settings, threadPool, testedActions);
|
||||
NodeClient client = new NodeClient(settings, threadPool);
|
||||
client.initialize(actions, () -> "test", null);
|
||||
client.initialize(actions, () -> "test", null,
|
||||
new NamedWriteableRegistry(Collections.emptyList()));
|
||||
return client;
|
||||
}
|
||||
|
||||
|
|
|
@ -84,11 +84,10 @@ import org.elasticsearch.indices.mapper.MapperRegistry;
|
|||
import org.elasticsearch.indices.recovery.RecoveryState;
|
||||
import org.elasticsearch.plugins.IndexStorePlugin;
|
||||
import org.elasticsearch.script.ScriptService;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
import org.elasticsearch.search.internal.ReaderContext;
|
||||
import org.elasticsearch.test.ClusterServiceUtils;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.test.IndexSettingsModule;
|
||||
import org.elasticsearch.test.TestSearchContext;
|
||||
import org.elasticsearch.test.engine.MockEngineFactory;
|
||||
import org.elasticsearch.threadpool.TestThreadPool;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
@ -288,9 +287,8 @@ public class IndexModuleTests extends ESTestCase {
|
|||
IndexModule module = createIndexModule(indexSettings, emptyAnalysisRegistry);
|
||||
AtomicBoolean executed = new AtomicBoolean(false);
|
||||
SearchOperationListener listener = new SearchOperationListener() {
|
||||
|
||||
@Override
|
||||
public void onNewContext(SearchContext context) {
|
||||
public void onNewReaderContext(ReaderContext readerContext) {
|
||||
executed.set(true);
|
||||
}
|
||||
};
|
||||
|
@ -303,9 +301,8 @@ public class IndexModuleTests extends ESTestCase {
|
|||
assertEquals(2, indexService.getSearchOperationListener().size());
|
||||
assertEquals(SearchSlowLog.class, indexService.getSearchOperationListener().get(0).getClass());
|
||||
assertSame(listener, indexService.getSearchOperationListener().get(1));
|
||||
|
||||
for (SearchOperationListener l : indexService.getSearchOperationListener()) {
|
||||
l.onNewContext(new TestSearchContext(null));
|
||||
l.onNewReaderContext(mock(ReaderContext.class));
|
||||
}
|
||||
assertTrue(executed.get());
|
||||
indexService.close("simon says", false);
|
||||
|
|
|
@ -18,6 +18,7 @@
|
|||
*/
|
||||
package org.elasticsearch.index.shard;
|
||||
|
||||
import org.elasticsearch.search.internal.ReaderContext;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.test.TestSearchContext;
|
||||
|
@ -33,6 +34,7 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
|
||||
import static org.hamcrest.Matchers.not;
|
||||
import static org.hamcrest.Matchers.sameInstance;
|
||||
import static org.mockito.Mockito.mock;
|
||||
|
||||
public class SearchOperationListenerTests extends ESTestCase {
|
||||
|
||||
|
@ -90,32 +92,32 @@ public class SearchOperationListenerTests extends ESTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void onNewContext(SearchContext context) {
|
||||
assertNotNull(context);
|
||||
public void onNewReaderContext(ReaderContext readerContext) {
|
||||
assertNotNull(readerContext);
|
||||
newContext.incrementAndGet();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFreeContext(SearchContext context) {
|
||||
assertNotNull(context);
|
||||
public void onFreeReaderContext(ReaderContext readerContext) {
|
||||
assertNotNull(readerContext);
|
||||
freeContext.incrementAndGet();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onNewScrollContext(SearchContext context) {
|
||||
assertNotNull(context);
|
||||
public void onNewScrollContext(ReaderContext readerContext) {
|
||||
assertNotNull(readerContext);
|
||||
newScrollContext.incrementAndGet();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFreeScrollContext(SearchContext context) {
|
||||
assertNotNull(context);
|
||||
public void onFreeScrollContext(ReaderContext readerContext) {
|
||||
assertNotNull(readerContext);
|
||||
freeScrollContext.incrementAndGet();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void validateSearchContext(SearchContext context, TransportRequest request) {
|
||||
assertNotNull(context);
|
||||
public void validateSearchContext(ReaderContext readerContext, TransportRequest request) {
|
||||
assertNotNull(readerContext);
|
||||
validateSearchContext.incrementAndGet();
|
||||
}
|
||||
};
|
||||
|
@ -216,7 +218,7 @@ public class SearchOperationListenerTests extends ESTestCase {
|
|||
assertEquals(0, freeScrollContext.get());
|
||||
assertEquals(0, validateSearchContext.get());
|
||||
|
||||
compositeListener.onNewContext(ctx);
|
||||
compositeListener.onNewReaderContext(mock(ReaderContext.class));
|
||||
assertEquals(2, preFetch.get());
|
||||
assertEquals(2, preQuery.get());
|
||||
assertEquals(2, failedFetch.get());
|
||||
|
@ -229,7 +231,7 @@ public class SearchOperationListenerTests extends ESTestCase {
|
|||
assertEquals(0, freeScrollContext.get());
|
||||
assertEquals(0, validateSearchContext.get());
|
||||
|
||||
compositeListener.onNewScrollContext(ctx);
|
||||
compositeListener.onNewScrollContext(mock(ReaderContext.class));
|
||||
assertEquals(2, preFetch.get());
|
||||
assertEquals(2, preQuery.get());
|
||||
assertEquals(2, failedFetch.get());
|
||||
|
@ -242,7 +244,7 @@ public class SearchOperationListenerTests extends ESTestCase {
|
|||
assertEquals(0, freeScrollContext.get());
|
||||
assertEquals(0, validateSearchContext.get());
|
||||
|
||||
compositeListener.onFreeContext(ctx);
|
||||
compositeListener.onFreeReaderContext(mock(ReaderContext.class));
|
||||
assertEquals(2, preFetch.get());
|
||||
assertEquals(2, preQuery.get());
|
||||
assertEquals(2, failedFetch.get());
|
||||
|
@ -255,7 +257,7 @@ public class SearchOperationListenerTests extends ESTestCase {
|
|||
assertEquals(0, freeScrollContext.get());
|
||||
assertEquals(0, validateSearchContext.get());
|
||||
|
||||
compositeListener.onFreeScrollContext(ctx);
|
||||
compositeListener.onFreeScrollContext(mock(ReaderContext.class));
|
||||
assertEquals(2, preFetch.get());
|
||||
assertEquals(2, preQuery.get());
|
||||
assertEquals(2, failedFetch.get());
|
||||
|
@ -269,10 +271,10 @@ public class SearchOperationListenerTests extends ESTestCase {
|
|||
assertEquals(0, validateSearchContext.get());
|
||||
|
||||
if (throwingListeners == 0) {
|
||||
compositeListener.validateSearchContext(ctx, Empty.INSTANCE);
|
||||
compositeListener.validateSearchContext(mock(ReaderContext.class), Empty.INSTANCE);
|
||||
} else {
|
||||
RuntimeException expected =
|
||||
expectThrows(RuntimeException.class, () -> compositeListener.validateSearchContext(ctx, Empty.INSTANCE));
|
||||
RuntimeException expected = expectThrows(RuntimeException.class,
|
||||
() -> compositeListener.validateSearchContext(mock(ReaderContext.class), Empty.INSTANCE));
|
||||
assertNull(expected.getMessage());
|
||||
assertEquals(throwingListeners - 1, expected.getSuppressed().length);
|
||||
if (throwingListeners > 1) {
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.elasticsearch.action.support.ActionFilters;
|
|||
import org.elasticsearch.action.support.TransportAction;
|
||||
import org.elasticsearch.client.node.NodeClient;
|
||||
import org.elasticsearch.common.bytes.BytesArray;
|
||||
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.util.concurrent.ThreadContext;
|
||||
import org.elasticsearch.common.xcontent.XContentType;
|
||||
|
@ -82,7 +83,7 @@ public class RestValidateQueryActionTests extends AbstractSearchTestCase {
|
|||
final Map<ActionType, TransportAction> actions = new HashMap<>();
|
||||
actions.put(ValidateQueryAction.INSTANCE, transportAction);
|
||||
|
||||
client.initialize(actions, () -> "local", null);
|
||||
client.initialize(actions, () -> "local", null, new NamedWriteableRegistry(Collections.emptyList()));
|
||||
controller.registerHandler(action);
|
||||
}
|
||||
|
||||
|
|
|
@ -31,7 +31,6 @@ import org.elasticsearch.Version;
|
|||
import org.elasticsearch.action.OriginalIndices;
|
||||
import org.elasticsearch.action.search.SearchType;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetadata;
|
||||
import org.elasticsearch.common.UUIDs;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.BigArrays;
|
||||
|
@ -51,8 +50,8 @@ import org.elasticsearch.index.shard.IndexShard;
|
|||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
|
||||
import org.elasticsearch.search.internal.AliasFilter;
|
||||
import org.elasticsearch.search.internal.ScrollContext;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.LegacyReaderContext;
|
||||
import org.elasticsearch.search.internal.ReaderContext;
|
||||
import org.elasticsearch.search.internal.ShardSearchRequest;
|
||||
import org.elasticsearch.search.rescore.RescoreContext;
|
||||
import org.elasticsearch.search.slice.SliceBuilder;
|
||||
|
@ -63,6 +62,8 @@ import org.elasticsearch.threadpool.ThreadPool;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.UUID;
|
||||
import java.util.function.Function;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.is;
|
||||
|
@ -82,9 +83,11 @@ public class DefaultSearchContextTests extends ESTestCase {
|
|||
when(shardSearchRequest.shardId()).thenReturn(shardId);
|
||||
when(shardSearchRequest.types()).thenReturn(new String[]{});
|
||||
|
||||
ThreadPool threadPool = new TestThreadPool(this.getClass().getName());
|
||||
IndexShard indexShard = mock(IndexShard.class);
|
||||
QueryCachingPolicy queryCachingPolicy = mock(QueryCachingPolicy.class);
|
||||
when(indexShard.getQueryCachingPolicy()).thenReturn(queryCachingPolicy);
|
||||
when(indexShard.getThreadPool()).thenReturn(threadPool);
|
||||
|
||||
int maxResultWindow = randomIntBetween(50, 100);
|
||||
int maxRescoreWindow = randomIntBetween(50, 100);
|
||||
|
@ -117,28 +120,49 @@ public class DefaultSearchContextTests extends ESTestCase {
|
|||
BigArrays bigArrays = new MockBigArrays(new MockPageCacheRecycler(Settings.EMPTY), new NoneCircuitBreakerService());
|
||||
|
||||
try (Directory dir = newDirectory();
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir);
|
||||
IndexReader reader = w.getReader();
|
||||
Engine.Searcher searcher = new Engine.Searcher("test", reader,
|
||||
IndexSearcher.getDefaultSimilarity(), IndexSearcher.getDefaultQueryCache(),
|
||||
IndexSearcher.getDefaultQueryCachingPolicy(), reader)) {
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir)) {
|
||||
|
||||
|
||||
final Supplier<Engine.SearcherSupplier> searcherSupplier = () -> new Engine.SearcherSupplier(Function.identity()) {
|
||||
@Override
|
||||
protected void doClose() {
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Engine.Searcher acquireSearcherInternal(String source) {
|
||||
try {
|
||||
IndexReader reader = w.getReader();
|
||||
return new Engine.Searcher("test", reader, IndexSearcher.getDefaultSimilarity(),
|
||||
IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), reader);
|
||||
} catch (IOException exc) {
|
||||
throw new AssertionError(exc);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
SearchShardTarget target = new SearchShardTarget("node", shardId, null, OriginalIndices.NONE);
|
||||
ReaderContext readerWithoutScroll = new ReaderContext(
|
||||
randomNonNegativeLong(), indexService, indexShard, searcherSupplier.get(), randomNonNegativeLong(), false);
|
||||
|
||||
DefaultSearchContext context1 = new DefaultSearchContext(new SearchContextId(UUIDs.randomBase64UUID(), 1L),
|
||||
shardSearchRequest, target, searcher, null, indexService, indexShard, bigArrays, null, timeout, null,
|
||||
false, Version.CURRENT);
|
||||
context1.from(300);
|
||||
DefaultSearchContext contextWithoutScroll = new DefaultSearchContext(readerWithoutScroll, shardSearchRequest, target, null,
|
||||
bigArrays, null, timeout, null, false, Version.CURRENT);
|
||||
contextWithoutScroll.from(300);
|
||||
contextWithoutScroll.close();
|
||||
|
||||
// resultWindow greater than maxResultWindow and scrollContext is null
|
||||
IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> context1.preProcess(false));
|
||||
IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> contextWithoutScroll.preProcess(false));
|
||||
assertThat(exception.getMessage(), equalTo("Result window is too large, from + size must be less than or equal to:"
|
||||
+ " [" + maxResultWindow + "] but was [310]. See the scroll api for a more efficient way to request large data sets. "
|
||||
+ "This limit can be set by changing the [" + IndexSettings.MAX_RESULT_WINDOW_SETTING.getKey()
|
||||
+ "] index level setting."));
|
||||
|
||||
// resultWindow greater than maxResultWindow and scrollContext isn't null
|
||||
context1.scrollContext(new ScrollContext());
|
||||
when(shardSearchRequest.scroll()).thenReturn(new Scroll(TimeValue.timeValueMillis(randomInt(1000))));
|
||||
ReaderContext readerContext = new LegacyReaderContext(
|
||||
randomNonNegativeLong(), indexService, indexShard, searcherSupplier.get(), shardSearchRequest, randomNonNegativeLong());
|
||||
DefaultSearchContext context1 = new DefaultSearchContext(readerContext, shardSearchRequest, target, null,
|
||||
bigArrays, null, timeout, null, false, Version.CURRENT);
|
||||
context1.from(300);
|
||||
exception = expectThrows(IllegalArgumentException.class, () -> context1.preProcess(false));
|
||||
assertThat(exception.getMessage(), equalTo("Batch size is too large, size must be less than or equal to: ["
|
||||
+ maxResultWindow + "] but was [310]. Scroll batch sizes cost as much memory as result windows so they are "
|
||||
|
@ -166,10 +190,12 @@ public class DefaultSearchContextTests extends ESTestCase {
|
|||
+ "to be rescored. This limit can be set by changing the [" + IndexSettings.MAX_RESCORE_WINDOW_SETTING.getKey()
|
||||
+ "] index level setting."));
|
||||
|
||||
readerContext.close();
|
||||
readerContext = new ReaderContext(
|
||||
randomNonNegativeLong(), indexService, indexShard, searcherSupplier.get(), randomNonNegativeLong(), false);
|
||||
// rescore is null but sliceBuilder is not null
|
||||
DefaultSearchContext context2 = new DefaultSearchContext(new SearchContextId(UUIDs.randomBase64UUID(), 2L),
|
||||
shardSearchRequest, target, searcher, null, indexService, indexShard, bigArrays, null, timeout, null,
|
||||
false, Version.CURRENT);
|
||||
DefaultSearchContext context2 = new DefaultSearchContext(readerContext, shardSearchRequest, target,
|
||||
null, bigArrays, null, timeout, null, false, Version.CURRENT);
|
||||
|
||||
SliceBuilder sliceBuilder = mock(SliceBuilder.class);
|
||||
int numSlices = maxSlicesPerScroll + randomIntBetween(1, 100);
|
||||
|
@ -185,9 +211,8 @@ public class DefaultSearchContextTests extends ESTestCase {
|
|||
when(shardSearchRequest.getAliasFilter()).thenReturn(AliasFilter.EMPTY);
|
||||
when(shardSearchRequest.indexBoost()).thenReturn(AbstractQueryBuilder.DEFAULT_BOOST);
|
||||
|
||||
DefaultSearchContext context3 = new DefaultSearchContext(new SearchContextId(UUIDs.randomBase64UUID(), 3L),
|
||||
shardSearchRequest, target, searcher, null, indexService, indexShard, bigArrays, null, timeout, null,
|
||||
false, Version.CURRENT);
|
||||
DefaultSearchContext context3 = new DefaultSearchContext(readerContext, shardSearchRequest, target, null,
|
||||
bigArrays, null, timeout, null, false, Version.CURRENT);
|
||||
ParsedQuery parsedQuery = ParsedQuery.parsedMatchAllQuery();
|
||||
context3.sliceBuilder(null).parsedQuery(parsedQuery).preProcess(false);
|
||||
assertEquals(context3.query(), context3.buildFilteredQuery(parsedQuery.query()));
|
||||
|
@ -196,15 +221,19 @@ public class DefaultSearchContextTests extends ESTestCase {
|
|||
when(queryShardContext.fieldMapper(anyString())).thenReturn(mock(MappedFieldType.class));
|
||||
when(shardSearchRequest.indexRoutings()).thenReturn(new String[0]);
|
||||
|
||||
DefaultSearchContext context4 = new DefaultSearchContext(new SearchContextId(UUIDs.randomBase64UUID(), 4L),
|
||||
shardSearchRequest, target, searcher, null, indexService, indexShard, bigArrays, null, timeout, null,
|
||||
false, Version.CURRENT);
|
||||
readerContext.close();
|
||||
readerContext = new ReaderContext(randomNonNegativeLong(), indexService, indexShard,
|
||||
searcherSupplier.get(), randomNonNegativeLong(), false);
|
||||
DefaultSearchContext context4 = new DefaultSearchContext(readerContext, shardSearchRequest, target, null, bigArrays, null,
|
||||
timeout, null, false, Version.CURRENT);
|
||||
context4.sliceBuilder(new SliceBuilder(1,2)).parsedQuery(parsedQuery).preProcess(false);
|
||||
Query query1 = context4.query();
|
||||
context4.sliceBuilder(new SliceBuilder(0,2)).parsedQuery(parsedQuery).preProcess(false);
|
||||
Query query2 = context4.query();
|
||||
assertTrue(query1 instanceof MatchNoDocsQuery || query2 instanceof MatchNoDocsQuery);
|
||||
|
||||
readerContext.close();
|
||||
threadPool.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -234,11 +263,23 @@ public class DefaultSearchContextTests extends ESTestCase {
|
|||
IndexSearcher.getDefaultSimilarity(), IndexSearcher.getDefaultQueryCache(),
|
||||
IndexSearcher.getDefaultQueryCachingPolicy(), reader)) {
|
||||
|
||||
Engine.SearcherSupplier searcherSupplier = new Engine.SearcherSupplier(Function.identity()) {
|
||||
@Override
|
||||
protected void doClose() {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Engine.Searcher acquireSearcherInternal(String source) {
|
||||
return searcher;
|
||||
}
|
||||
};
|
||||
SearchShardTarget target = new SearchShardTarget("node", shardId, null, OriginalIndices.NONE);
|
||||
DefaultSearchContext context = new DefaultSearchContext(new SearchContextId(UUIDs.randomBase64UUID(), 1L), shardSearchRequest,
|
||||
target, searcher, null, indexService, indexShard, bigArrays, null, timeout, null, false, Version.CURRENT);
|
||||
ReaderContext readerContext = new ReaderContext(
|
||||
randomNonNegativeLong(), indexService, indexShard, searcherSupplier, randomNonNegativeLong(), false);
|
||||
|
||||
DefaultSearchContext context = new DefaultSearchContext(
|
||||
readerContext, shardSearchRequest, target, null, bigArrays, null, timeout, null, false, Version.CURRENT);
|
||||
assertThat(context.searcher().hasCancellations(), is(false));
|
||||
context.searcher().addQueryCancellation(() -> {});
|
||||
assertThat(context.searcher().hasCancellations(), is(true));
|
||||
|
|
|
@ -49,6 +49,7 @@ import org.elasticsearch.index.IndexModule;
|
|||
import org.elasticsearch.index.IndexNotFoundException;
|
||||
import org.elasticsearch.index.IndexService;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
import org.elasticsearch.index.engine.Engine;
|
||||
import org.elasticsearch.index.query.AbstractQueryBuilder;
|
||||
import org.elasticsearch.index.query.MatchAllQueryBuilder;
|
||||
import org.elasticsearch.index.query.MatchNoneQueryBuilder;
|
||||
|
@ -79,12 +80,14 @@ import org.elasticsearch.search.builder.SearchSourceBuilder;
|
|||
import org.elasticsearch.search.fetch.FetchSearchResult;
|
||||
import org.elasticsearch.search.fetch.ShardFetchRequest;
|
||||
import org.elasticsearch.search.internal.AliasFilter;
|
||||
import org.elasticsearch.search.internal.ReaderContext;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchRequest;
|
||||
import org.elasticsearch.search.query.QuerySearchResult;
|
||||
import org.elasticsearch.search.suggest.SuggestBuilder;
|
||||
import org.elasticsearch.test.ESSingleNodeTestCase;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.junit.Before;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -173,17 +176,6 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
|||
@Override
|
||||
public void onIndexModule(IndexModule indexModule) {
|
||||
indexModule.addSearchOperationListener(new SearchOperationListener() {
|
||||
@Override
|
||||
public void onNewContext(SearchContext context) {
|
||||
if (context.query() != null) {
|
||||
if ("throttled_threadpool_index".equals(context.indexShard().shardId().getIndex().getName())) {
|
||||
assertThat(Thread.currentThread().getName(), startsWith("elasticsearch[node_s_0][search_throttled]"));
|
||||
} else {
|
||||
assertThat(Thread.currentThread().getName(), startsWith("elasticsearch[node_s_0][search]"));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFetchPhase(SearchContext context, long tookInNanos) {
|
||||
if ("throttled_threadpool_index".equals(context.indexShard().shardId().getIndex().getName())) {
|
||||
|
@ -322,6 +314,7 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
|||
new ShardSearchRequest(OriginalIndices.NONE, useScroll ? scrollSearchRequest : searchRequest,
|
||||
indexShard.shardId(), 1,
|
||||
new AliasFilter(null, Strings.EMPTY_ARRAY), 1.0f, -1, null, null),
|
||||
true,
|
||||
new SearchShardTask(123L, "", "", "", null, Collections.emptyMap()), result);
|
||||
SearchPhaseResult searchPhaseResult = result.get();
|
||||
IntArrayList intCursors = new IntArrayList(1);
|
||||
|
@ -332,7 +325,7 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
|||
listener.get();
|
||||
if (useScroll) {
|
||||
// have to free context since this test does not remove the index from IndicesService.
|
||||
service.freeContext(searchPhaseResult.getContextId());
|
||||
service.freeReaderContext(searchPhaseResult.getContextId());
|
||||
}
|
||||
} catch (ExecutionException ex) {
|
||||
assertThat(ex.getCause(), instanceOf(RuntimeException.class));
|
||||
|
@ -341,7 +334,7 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
|||
} catch (AlreadyClosedException ex) {
|
||||
throw ex;
|
||||
} catch (IllegalStateException ex) {
|
||||
assertEquals("search context is already closed can't increment refCount current count [0]", ex.getMessage());
|
||||
assertEquals("reader_context is already closed can't increment refCount current count [0]", ex.getMessage());
|
||||
} catch (SearchContextMissingException ex) {
|
||||
// that's fine
|
||||
}
|
||||
|
@ -389,7 +382,7 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
|||
new ShardSearchRequest(OriginalIndices.NONE, useScroll ? scrollSearchRequest : searchRequest,
|
||||
new ShardId(resolveIndex("index"), 0), 1,
|
||||
new AliasFilter(null, Strings.EMPTY_ARRAY), 1.0f, -1, null, null),
|
||||
new SearchShardTask(123L, "", "", "", null, Collections.emptyMap()), result);
|
||||
randomBoolean(), new SearchShardTask(123L, "", "", "", null, Collections.emptyMap()), result);
|
||||
|
||||
try {
|
||||
result.get();
|
||||
|
@ -414,42 +407,34 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
|||
final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index"));
|
||||
final IndexShard indexShard = indexService.getShard(0);
|
||||
SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true);
|
||||
final SearchContext contextWithDefaultTimeout = service.createContext(
|
||||
new ShardSearchRequest(
|
||||
OriginalIndices.NONE,
|
||||
searchRequest,
|
||||
indexShard.shardId(),
|
||||
1,
|
||||
new AliasFilter(null, Strings.EMPTY_ARRAY),
|
||||
1.0f, -1, null, null
|
||||
), null);
|
||||
try {
|
||||
final ShardSearchRequest requestWithDefaultTimeout = new ShardSearchRequest(
|
||||
OriginalIndices.NONE,
|
||||
searchRequest,
|
||||
indexShard.shardId(),
|
||||
1,
|
||||
new AliasFilter(null, Strings.EMPTY_ARRAY),
|
||||
1.0f, -1, null, null);
|
||||
|
||||
try (ReaderContext reader = createReaderContext(indexService, indexShard);
|
||||
SearchContext contextWithDefaultTimeout = service.createContext(reader, requestWithDefaultTimeout, null, randomBoolean())) {
|
||||
// the search context should inherit the default timeout
|
||||
assertThat(contextWithDefaultTimeout.timeout(), equalTo(TimeValue.timeValueSeconds(5)));
|
||||
} finally {
|
||||
contextWithDefaultTimeout.decRef();
|
||||
service.freeContext(contextWithDefaultTimeout.id());
|
||||
}
|
||||
|
||||
final long seconds = randomIntBetween(6, 10);
|
||||
searchRequest.source(new SearchSourceBuilder().timeout(TimeValue.timeValueSeconds(seconds)));
|
||||
final SearchContext context = service.createContext(
|
||||
new ShardSearchRequest(
|
||||
OriginalIndices.NONE,
|
||||
searchRequest,
|
||||
indexShard.shardId(),
|
||||
1,
|
||||
new AliasFilter(null, Strings.EMPTY_ARRAY),
|
||||
1.0f, -1, null, null
|
||||
), null);
|
||||
try {
|
||||
final ShardSearchRequest requestWithCustomTimeout = new ShardSearchRequest(
|
||||
OriginalIndices.NONE,
|
||||
searchRequest,
|
||||
indexShard.shardId(),
|
||||
1,
|
||||
new AliasFilter(null, Strings.EMPTY_ARRAY),
|
||||
1.0f, -1, null, null);
|
||||
try (ReaderContext reader = createReaderContext(indexService, indexShard);
|
||||
SearchContext context = service.createContext(reader, requestWithCustomTimeout, null, randomBoolean())) {
|
||||
// the search context should inherit the query timeout
|
||||
assertThat(context.timeout(), equalTo(TimeValue.timeValueSeconds(seconds)));
|
||||
} finally {
|
||||
context.decRef();
|
||||
service.freeContext(context.id());
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -469,19 +454,20 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
|||
for (int i = 0; i < indexService.getIndexSettings().getMaxDocvalueFields(); i++) {
|
||||
searchSourceBuilder.docValueField("field" + i);
|
||||
}
|
||||
try (SearchContext context = service.createContext(
|
||||
new ShardSearchRequest(OriginalIndices.NONE,
|
||||
searchRequest, indexShard.shardId(), 1,
|
||||
new AliasFilter(null, Strings.EMPTY_ARRAY), 1.0f, -1, null, null
|
||||
), null)) {
|
||||
final ShardSearchRequest request = new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 1,
|
||||
new AliasFilter(null, Strings.EMPTY_ARRAY), 1.0f, -1, null, null);
|
||||
try (ReaderContext reader = createReaderContext(indexService, indexShard);
|
||||
SearchContext context = service.createContext(reader, request, null, randomBoolean())) {
|
||||
assertNotNull(context);
|
||||
searchSourceBuilder.docValueField("one_field_too_much");
|
||||
}
|
||||
searchSourceBuilder.docValueField("one_field_too_much");
|
||||
try (ReaderContext reader = createReaderContext(indexService, indexShard)) {
|
||||
IllegalArgumentException ex = expectThrows(IllegalArgumentException.class,
|
||||
() -> service.createContext(new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 1,
|
||||
new AliasFilter(null, Strings.EMPTY_ARRAY), 1.0f, -1, null, null), null));
|
||||
() -> service.createContext(reader, request, null, randomBoolean()));
|
||||
assertEquals(
|
||||
"Trying to retrieve too many docvalue_fields. Must be less than or equal to: [100] but was [101]. "
|
||||
+ "This limit can be set by changing the [index.max_docvalue_fields_search] index level setting.", ex.getMessage());
|
||||
+ "This limit can be set by changing the [index.max_docvalue_fields_search] index level setting.",
|
||||
ex.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -504,15 +490,17 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
|||
searchSourceBuilder.scriptField("field" + i,
|
||||
new Script(ScriptType.INLINE, MockScriptEngine.NAME, CustomScriptPlugin.DUMMY_SCRIPT, Collections.emptyMap()));
|
||||
}
|
||||
try (SearchContext context = service.createContext(new ShardSearchRequest(OriginalIndices.NONE, searchRequest,
|
||||
indexShard.shardId(), 1, new AliasFilter(null, Strings.EMPTY_ARRAY),
|
||||
1.0f, -1, null, null), null)) {
|
||||
assertNotNull(context);
|
||||
final ShardSearchRequest request = new ShardSearchRequest(OriginalIndices.NONE, searchRequest,
|
||||
indexShard.shardId(), 1, new AliasFilter(null, Strings.EMPTY_ARRAY), 1.0f, -1, null, null);
|
||||
|
||||
try(ReaderContext reader = createReaderContext(indexService, indexShard)) {
|
||||
try (SearchContext context = service.createContext(reader, request, null, randomBoolean())) {
|
||||
assertNotNull(context);
|
||||
}
|
||||
searchSourceBuilder.scriptField("anotherScriptField",
|
||||
new Script(ScriptType.INLINE, MockScriptEngine.NAME, CustomScriptPlugin.DUMMY_SCRIPT, Collections.emptyMap()));
|
||||
IllegalArgumentException ex = expectThrows(IllegalArgumentException.class,
|
||||
() -> service.createContext(new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 1,
|
||||
new AliasFilter(null, Strings.EMPTY_ARRAY), 1.0f, -1, null, null), null));
|
||||
() -> service.createContext(reader, request, null, randomBoolean()));
|
||||
assertEquals(
|
||||
"Trying to retrieve too many script_fields. Must be less than or equal to: [" + maxScriptFields + "] but was ["
|
||||
+ (maxScriptFields + 1)
|
||||
|
@ -534,17 +522,19 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
|||
searchSourceBuilder.scriptField("field" + 0,
|
||||
new Script(ScriptType.INLINE, MockScriptEngine.NAME, CustomScriptPlugin.DUMMY_SCRIPT, Collections.emptyMap()));
|
||||
searchSourceBuilder.size(0);
|
||||
try (SearchContext context = service.createContext(new ShardSearchRequest(OriginalIndices.NONE,
|
||||
final ShardSearchRequest request = new ShardSearchRequest(OriginalIndices.NONE,
|
||||
searchRequest, indexShard.shardId(), 1, new AliasFilter(null, Strings.EMPTY_ARRAY),
|
||||
1.0f, -1, null, null), null)) {
|
||||
assertEquals(0, context.scriptFields().fields().size());
|
||||
1.0f, -1, null, null);
|
||||
try (ReaderContext reader = createReaderContext(indexService, indexShard);
|
||||
SearchContext context = service.createContext(reader, request, null, randomBoolean())) {
|
||||
assertEquals(0, context.scriptFields().fields().size());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* test that creating more than the allowed number of scroll contexts throws an exception
|
||||
*/
|
||||
public void testMaxOpenScrollContexts() throws RuntimeException {
|
||||
public void testMaxOpenScrollContexts() throws Exception {
|
||||
createIndex("index");
|
||||
client().prepareIndex("index", "type", "1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get();
|
||||
|
||||
|
@ -570,8 +560,10 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
|||
client().prepareSearch("index").setSize(1).setScroll("1m").get();
|
||||
}
|
||||
|
||||
final ShardScrollRequestTest request = new ShardScrollRequestTest(indexShard.shardId());
|
||||
ElasticsearchException ex = expectThrows(ElasticsearchException.class,
|
||||
() -> service.createAndPutContext(new ShardScrollRequestTest(indexShard.shardId()), null));
|
||||
() -> service.createAndPutReaderContext(
|
||||
request, indexService, indexShard, indexShard.acquireSearcherSupplier(), randomBoolean()));
|
||||
assertEquals(
|
||||
"Trying to create too many scroll contexts. Must be less than or equal to: [" +
|
||||
SearchService.MAX_OPEN_SCROLL_CONTEXT.get(Settings.EMPTY) + "]. " +
|
||||
|
@ -584,7 +576,8 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
|||
public void testOpenScrollContextsConcurrently() throws Exception {
|
||||
createIndex("index");
|
||||
final IndicesService indicesService = getInstanceFromNode(IndicesService.class);
|
||||
final IndexShard indexShard = indicesService.indexServiceSafe(resolveIndex("index")).getShard(0);
|
||||
final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index"));
|
||||
final IndexShard indexShard = indexService.getShard(0);
|
||||
|
||||
final int maxScrollContexts = SearchService.MAX_OPEN_SCROLL_CONTEXT.get(Settings.EMPTY);
|
||||
final SearchService searchService = getInstanceFromNode(SearchService.class);
|
||||
|
@ -596,8 +589,10 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
|||
try {
|
||||
latch.await();
|
||||
for (; ; ) {
|
||||
final Engine.SearcherSupplier reader = indexShard.acquireSearcherSupplier();
|
||||
try {
|
||||
searchService.createAndPutContext(new ShardScrollRequestTest(indexShard.shardId()), null);
|
||||
searchService.createAndPutReaderContext(
|
||||
new ShardScrollRequestTest(indexShard.shardId()), indexService, indexShard, reader, true);
|
||||
} catch (ElasticsearchException e) {
|
||||
assertThat(e.getMessage(), equalTo(
|
||||
"Trying to create too many scroll contexts. Must be less than or equal to: " +
|
||||
|
@ -690,7 +685,7 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
public void testCanMatch() throws IOException, InterruptedException {
|
||||
public void testCanMatch() throws Exception {
|
||||
createIndex("index");
|
||||
final SearchService service = getInstanceFromNode(SearchService.class);
|
||||
final IndicesService indicesService = getInstanceFromNode(IndicesService.class);
|
||||
|
@ -741,7 +736,7 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
|||
|
||||
CountDownLatch latch = new CountDownLatch(1);
|
||||
SearchShardTask task = new SearchShardTask(123L, "", "", "", null, Collections.emptyMap());
|
||||
service.executeQueryPhase(request, task, new ActionListener<SearchPhaseResult>() {
|
||||
service.executeQueryPhase(request, randomBoolean(), task, new ActionListener<SearchPhaseResult>() {
|
||||
@Override
|
||||
public void onResponse(SearchPhaseResult searchPhaseResult) {
|
||||
try {
|
||||
|
@ -890,18 +885,18 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
|||
final IndexService indexService = createIndex(index);
|
||||
final SearchService service = getInstanceFromNode(SearchService.class);
|
||||
final ShardId shardId = new ShardId(indexService.index(), 0);
|
||||
IndexShard indexShard = indexService.getShard(0);
|
||||
|
||||
NullPointerException e = expectThrows(NullPointerException.class,
|
||||
() -> service.createContext(
|
||||
new ShardSearchRequest(shardId, null, 0, AliasFilter.EMPTY) {
|
||||
@Override
|
||||
public SearchType searchType() {
|
||||
// induce an artificial NPE
|
||||
throw new NullPointerException("expected");
|
||||
}
|
||||
}, null));
|
||||
assertEquals("expected", e.getMessage());
|
||||
final ShardSearchRequest request = new ShardSearchRequest(shardId, new String[0], 0, null) {
|
||||
@Override
|
||||
public SearchType searchType() {
|
||||
// induce an artificial NPE
|
||||
throw new NullPointerException("expected");
|
||||
}
|
||||
};
|
||||
try (ReaderContext reader = createReaderContext(indexService, indexService.getShard(shardId.id()))) {
|
||||
NullPointerException e = expectThrows(NullPointerException.class,
|
||||
() -> service.createContext(reader, request, null, randomBoolean()));
|
||||
assertEquals("expected", e.getMessage());
|
||||
}
|
||||
assertEquals("should have 2 store refs (IndexService + InternalEngine)", 2, indexService.getShard(0).store().refCount());
|
||||
}
|
||||
|
||||
|
@ -923,7 +918,7 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
|||
{
|
||||
CountDownLatch latch = new CountDownLatch(1);
|
||||
shardRequest.source().query(new MatchAllQueryBuilder());
|
||||
service.executeQueryPhase(shardRequest, task, new ActionListener<SearchPhaseResult>() {
|
||||
service.executeQueryPhase(shardRequest, randomBoolean(), task, new ActionListener<SearchPhaseResult>() {
|
||||
@Override
|
||||
public void onResponse(SearchPhaseResult result) {
|
||||
try {
|
||||
|
@ -953,7 +948,7 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
|||
{
|
||||
CountDownLatch latch = new CountDownLatch(1);
|
||||
shardRequest.source().query(new MatchNoneQueryBuilder());
|
||||
service.executeQueryPhase(shardRequest, task, new ActionListener<SearchPhaseResult>() {
|
||||
service.executeQueryPhase(shardRequest, randomBoolean(), task, new ActionListener<SearchPhaseResult>() {
|
||||
@Override
|
||||
public void onResponse(SearchPhaseResult result) {
|
||||
try {
|
||||
|
@ -983,7 +978,7 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
|||
{
|
||||
CountDownLatch latch = new CountDownLatch(1);
|
||||
shardRequest.canReturnNullResponseIfMatchNoDocs(true);
|
||||
service.executeQueryPhase(shardRequest, task, new ActionListener<SearchPhaseResult>() {
|
||||
service.executeQueryPhase(shardRequest, randomBoolean(), task, new ActionListener<SearchPhaseResult>() {
|
||||
@Override
|
||||
public void onResponse(SearchPhaseResult result) {
|
||||
try {
|
||||
|
@ -1046,32 +1041,55 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
|||
IndicesService indicesService = getInstanceFromNode(IndicesService.class);
|
||||
IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index"));
|
||||
IndexShard indexShard = indexService.getShard(0);
|
||||
ShardSearchRequest shardSearchRequest = new ShardSearchRequest(
|
||||
OriginalIndices.NONE, new SearchRequest().allowPartialSearchResults(true),
|
||||
indexShard.shardId(), 1, new AliasFilter(null, Strings.EMPTY_ARRAY), 1.0f, -1, null, null);
|
||||
List<SearchContextId> contextIds = new ArrayList<>();
|
||||
List<ShardSearchContextId> contextIds = new ArrayList<>();
|
||||
int numContexts = randomIntBetween(1, 10);
|
||||
for (int i = 0; i < numContexts; i++) {
|
||||
final SearchContext searchContext = searchService.createContext(shardSearchRequest, null);
|
||||
assertThat(searchContext.id().getId(), equalTo((long) (i + 1)));
|
||||
searchService.putContext(searchContext);
|
||||
contextIds.add(searchContext.id());
|
||||
}
|
||||
assertThat(searchService.getActiveContexts(), equalTo(contextIds.size()));
|
||||
while (contextIds.isEmpty() == false) {
|
||||
final SearchContextId contextId = randomFrom(contextIds);
|
||||
assertFalse(searchService.freeContext(new SearchContextId(UUIDs.randomBase64UUID(), contextId.getId())));
|
||||
assertThat(searchService.getActiveContexts(), equalTo(contextIds.size()));
|
||||
if (randomBoolean()) {
|
||||
assertTrue(searchService.freeContext(contextId));
|
||||
} else {
|
||||
assertTrue(searchService.freeContext((new SearchContextId("", contextId.getId()))));
|
||||
CountDownLatch latch = new CountDownLatch(1);
|
||||
indexShard.getThreadPool().executor(ThreadPool.Names.SEARCH).execute(() -> {
|
||||
try {
|
||||
for (int i = 0; i < numContexts; i++) {
|
||||
ShardSearchRequest request = new ShardSearchRequest(
|
||||
OriginalIndices.NONE, new SearchRequest().allowPartialSearchResults(true),
|
||||
indexShard.shardId(), 1, new AliasFilter(null, Strings.EMPTY_ARRAY), 1.0f, -1, null, null);
|
||||
final ReaderContext context = searchService.createAndPutReaderContext(request, indexService, indexShard,
|
||||
indexShard.acquireSearcherSupplier(), randomBoolean());
|
||||
assertThat(context.id().getId(), equalTo((long) (i + 1)));
|
||||
contextIds.add(context.id());
|
||||
}
|
||||
assertThat(searchService.getActiveContexts(), equalTo(contextIds.size()));
|
||||
while (contextIds.isEmpty() == false) {
|
||||
final ShardSearchContextId contextId = randomFrom(contextIds);
|
||||
assertFalse(searchService.freeReaderContext(new ShardSearchContextId(UUIDs.randomBase64UUID(), contextId.getId())));
|
||||
assertThat(searchService.getActiveContexts(), equalTo(contextIds.size()));
|
||||
if (randomBoolean()) {
|
||||
assertTrue(searchService.freeReaderContext(contextId));
|
||||
} else {
|
||||
assertTrue(searchService.freeReaderContext((new ShardSearchContextId("", contextId.getId()))));
|
||||
}
|
||||
contextIds.remove(contextId);
|
||||
assertThat(searchService.getActiveContexts(), equalTo(contextIds.size()));
|
||||
assertFalse(searchService.freeReaderContext(new ShardSearchContextId("", contextId.getId())));
|
||||
assertFalse(searchService.freeReaderContext(contextId));
|
||||
assertThat(searchService.getActiveContexts(), equalTo(contextIds.size()));
|
||||
}
|
||||
} finally {
|
||||
latch.countDown();
|
||||
}
|
||||
contextIds.remove(contextId);
|
||||
assertThat(searchService.getActiveContexts(), equalTo(contextIds.size()));
|
||||
assertFalse(searchService.freeContext(new SearchContextId("", contextId.getId())));
|
||||
assertFalse(searchService.freeContext(contextId));
|
||||
assertThat(searchService.getActiveContexts(), equalTo(contextIds.size()));
|
||||
}
|
||||
});
|
||||
latch.await();
|
||||
}
|
||||
|
||||
public void testOpenReaderContext() {
|
||||
createIndex("index");
|
||||
SearchService searchService = getInstanceFromNode(SearchService.class);
|
||||
PlainActionFuture<ShardSearchContextId> future = new PlainActionFuture<>();
|
||||
searchService.openReaderContext(new ShardId(resolveIndex("index"), 0), TimeValue.timeValueMinutes(between(1, 10)), future);
|
||||
future.actionGet();
|
||||
assertThat(searchService.getActiveContexts(), equalTo(1));
|
||||
assertTrue(searchService.freeReaderContext(future.actionGet()));
|
||||
}
|
||||
|
||||
private ReaderContext createReaderContext(IndexService indexService, IndexShard indexShard) {
|
||||
return new ReaderContext(randomNonNegativeLong(), indexService, indexShard,
|
||||
indexShard.acquireSearcherSupplier(), randomNonNegativeLong(), false);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,36 +0,0 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search.internal;
|
||||
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
public class ScrollContextTests extends ESTestCase {
|
||||
|
||||
public void testStoringObjectsInScrollContext() {
|
||||
final ScrollContext scrollContext = new ScrollContext();
|
||||
final String key = randomAlphaOfLengthBetween(1, 16);
|
||||
assertNull(scrollContext.getFromContext(key));
|
||||
|
||||
final String value = randomAlphaOfLength(6);
|
||||
scrollContext.putInContext(key, value);
|
||||
|
||||
assertEquals(value, scrollContext.getFromContext(key));
|
||||
}
|
||||
}
|
|
@ -330,13 +330,12 @@ public class QueryPhaseTests extends IndexShardTestCase {
|
|||
}
|
||||
w.close();
|
||||
IndexReader reader = DirectoryReader.open(dir);
|
||||
TestSearchContext context = new TestSearchContext(null, indexShard, newContextSearcher(reader));
|
||||
context.parsedQuery(new ParsedQuery(new MatchAllDocsQuery()));
|
||||
ScrollContext scrollContext = new ScrollContext();
|
||||
TestSearchContext context = new TestSearchContext(null, indexShard, newContextSearcher(reader), scrollContext);
|
||||
context.parsedQuery(new ParsedQuery(new MatchAllDocsQuery()));
|
||||
scrollContext.lastEmittedDoc = null;
|
||||
scrollContext.maxScore = Float.NaN;
|
||||
scrollContext.totalHits = null;
|
||||
context.scrollContext(scrollContext);
|
||||
context.setTask(new SearchShardTask(123L, "", "", "", null, Collections.emptyMap()));
|
||||
int size = randomIntBetween(2, 5);
|
||||
context.setSize(size);
|
||||
|
@ -593,13 +592,12 @@ public class QueryPhaseTests extends IndexShardTestCase {
|
|||
// search sort is a prefix of the index sort
|
||||
searchSortAndFormats.add(new SortAndFormats(new Sort(indexSort.getSort()[0]), new DocValueFormat[]{DocValueFormat.RAW}));
|
||||
for (SortAndFormats searchSortAndFormat : searchSortAndFormats) {
|
||||
TestSearchContext context = new TestSearchContext(null, indexShard, newContextSearcher(reader));
|
||||
context.parsedQuery(new ParsedQuery(new MatchAllDocsQuery()));
|
||||
ScrollContext scrollContext = new ScrollContext();
|
||||
TestSearchContext context = new TestSearchContext(null, indexShard, newContextSearcher(reader), scrollContext);
|
||||
context.parsedQuery(new ParsedQuery(new MatchAllDocsQuery()));
|
||||
scrollContext.lastEmittedDoc = null;
|
||||
scrollContext.maxScore = Float.NaN;
|
||||
scrollContext.totalHits = null;
|
||||
context.scrollContext(scrollContext);
|
||||
context.setTask(new SearchShardTask(123L, "", "", "", null, Collections.emptyMap()));
|
||||
context.setSize(10);
|
||||
context.sort(searchSortAndFormat);
|
||||
|
|
|
@ -24,6 +24,10 @@ import org.apache.lucene.search.TopDocs;
|
|||
import org.apache.lucene.search.TotalHits;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.OriginalIndices;
|
||||
import org.elasticsearch.action.OriginalIndicesTests;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.UUIDs;
|
||||
import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput;
|
||||
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
|
@ -36,7 +40,9 @@ import org.elasticsearch.search.SearchShardTarget;
|
|||
import org.elasticsearch.search.aggregations.Aggregations;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregations;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregationsTests;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.AliasFilter;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchRequest;
|
||||
import org.elasticsearch.search.suggest.SuggestTests;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
|
@ -56,8 +62,11 @@ public class QuerySearchResultTests extends ESTestCase {
|
|||
|
||||
private static QuerySearchResult createTestInstance() throws Exception {
|
||||
ShardId shardId = new ShardId("index", "uuid", randomInt());
|
||||
QuerySearchResult result = new QuerySearchResult(new SearchContextId("", randomLong()),
|
||||
new SearchShardTarget("node", shardId, null, OriginalIndices.NONE));
|
||||
SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(randomBoolean());
|
||||
ShardSearchRequest shardSearchRequest = new ShardSearchRequest(OriginalIndicesTests.randomOriginalIndices(), searchRequest,
|
||||
shardId, 1, new AliasFilter(null, Strings.EMPTY_ARRAY), 1.0f, randomNonNegativeLong(), null, new String[0]);
|
||||
QuerySearchResult result = new QuerySearchResult(new ShardSearchContextId(UUIDs.base64UUID(), randomLong()),
|
||||
new SearchShardTarget("node", shardId, null, OriginalIndices.NONE), shardSearchRequest);
|
||||
if (randomBoolean()) {
|
||||
result.terminatedEarly(randomBoolean());
|
||||
}
|
||||
|
@ -77,7 +86,7 @@ public class QuerySearchResultTests extends ESTestCase {
|
|||
public void testSerialization() throws Exception {
|
||||
QuerySearchResult querySearchResult = createTestInstance();
|
||||
QuerySearchResult deserialized = copyWriteable(querySearchResult, namedWriteableRegistry, QuerySearchResult::new);
|
||||
assertEquals(querySearchResult.getContextId(), deserialized.getContextId());
|
||||
assertEquals(querySearchResult.getContextId().getId(), deserialized.getContextId().getId());
|
||||
assertNull(deserialized.getSearchShardTarget());
|
||||
assertEquals(querySearchResult.topDocs().maxScore, deserialized.topDocs().maxScore, 0f);
|
||||
assertEquals(querySearchResult.topDocs().topDocs.totalHits, deserialized.topDocs().topDocs.totalHits);
|
||||
|
|
|
@ -1616,7 +1616,7 @@ public class SnapshotResiliencyTests extends ESTestCase {
|
|||
actions.put(SearchAction.INSTANCE,
|
||||
new TransportSearchAction(client, threadPool, transportService, searchService,
|
||||
searchTransportService, searchPhaseController, clusterService,
|
||||
actionFilters, indexNameExpressionResolver));
|
||||
actionFilters, indexNameExpressionResolver, namedWriteableRegistry));
|
||||
actions.put(RestoreSnapshotAction.INSTANCE,
|
||||
new TransportRestoreSnapshotAction(transportService, clusterService, threadPool, restoreService, actionFilters,
|
||||
indexNameExpressionResolver));
|
||||
|
@ -1654,7 +1654,8 @@ public class SnapshotResiliencyTests extends ESTestCase {
|
|||
transportService, clusterService, threadPool,
|
||||
snapshotsService, actionFilters, indexNameExpressionResolver
|
||||
));
|
||||
client.initialize(actions, () -> clusterService.localNode().getId(), transportService.getRemoteClusterService());
|
||||
client.initialize(actions, () -> clusterService.localNode().getId(), transportService.getRemoteClusterService(),
|
||||
new NamedWriteableRegistry(Collections.emptyList()));
|
||||
}
|
||||
|
||||
private Repository.Factory getRepoFactory(Environment environment) {
|
||||
|
|
|
@ -27,7 +27,7 @@ import org.elasticsearch.node.MockNode;
|
|||
import org.elasticsearch.plugins.Plugin;
|
||||
import org.elasticsearch.script.ScriptService;
|
||||
import org.elasticsearch.search.fetch.FetchPhase;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
import org.elasticsearch.search.internal.ReaderContext;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
||||
import java.util.HashMap;
|
||||
|
@ -41,13 +41,13 @@ public class MockSearchService extends SearchService {
|
|||
*/
|
||||
public static class TestPlugin extends Plugin {}
|
||||
|
||||
private static final Map<SearchContext, Throwable> ACTIVE_SEARCH_CONTEXTS = new ConcurrentHashMap<>();
|
||||
private static final Map<ReaderContext, Throwable> ACTIVE_SEARCH_CONTEXTS = new ConcurrentHashMap<>();
|
||||
|
||||
private Consumer<SearchContext> onPutContext = context -> {};
|
||||
private Consumer<ReaderContext> onPutContext = context -> {};
|
||||
|
||||
/** Throw an {@link AssertionError} if there are still in-flight contexts. */
|
||||
public static void assertNoInFlightContext() {
|
||||
final Map<SearchContext, Throwable> copy = new HashMap<>(ACTIVE_SEARCH_CONTEXTS);
|
||||
final Map<ReaderContext, Throwable> copy = new HashMap<>(ACTIVE_SEARCH_CONTEXTS);
|
||||
if (copy.isEmpty() == false) {
|
||||
throw new AssertionError(
|
||||
"There are still [" + copy.size()
|
||||
|
@ -59,14 +59,14 @@ public class MockSearchService extends SearchService {
|
|||
/**
|
||||
* Add an active search context to the list of tracked contexts. Package private for testing.
|
||||
*/
|
||||
static void addActiveContext(SearchContext context) {
|
||||
static void addActiveContext(ReaderContext context) {
|
||||
ACTIVE_SEARCH_CONTEXTS.put(context, new RuntimeException(context.toString()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear an active search context from the list of tracked contexts. Package private for testing.
|
||||
*/
|
||||
static void removeActiveContext(SearchContext context) {
|
||||
static void removeActiveContext(ReaderContext context) {
|
||||
ACTIVE_SEARCH_CONTEXTS.remove(context);
|
||||
}
|
||||
|
||||
|
@ -77,22 +77,22 @@ public class MockSearchService extends SearchService {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected void putContext(SearchContext context) {
|
||||
protected void putReaderContext(ReaderContext context) {
|
||||
onPutContext.accept(context);
|
||||
addActiveContext(context);
|
||||
super.putContext(context);
|
||||
super.putReaderContext(context);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected SearchContext removeContext(long id) {
|
||||
final SearchContext removed = super.removeContext(id);
|
||||
protected ReaderContext removeReaderContext(long id) {
|
||||
final ReaderContext removed = super.removeReaderContext(id);
|
||||
if (removed != null) {
|
||||
removeActiveContext(removed);
|
||||
}
|
||||
return removed;
|
||||
}
|
||||
|
||||
public void setOnPutContext(Consumer<SearchContext> onPutContext) {
|
||||
public void setOnPutContext(Consumer<ReaderContext> onPutContext) {
|
||||
this.onPutContext = onPutContext;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -350,7 +350,7 @@ public abstract class AggregatorTestCase extends ESTestCase {
|
|||
* close their sub-aggregations. This is fairly similar to what the production code does. */
|
||||
releasables.add((Releasable) invocation.getArguments()[0]);
|
||||
return null;
|
||||
}).when(searchContext).addReleasable(anyObject(), anyObject());
|
||||
}).when(searchContext).addReleasable(anyObject());
|
||||
return searchContext;
|
||||
}
|
||||
|
||||
|
|
|
@ -51,9 +51,10 @@ import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
|
|||
import org.elasticsearch.search.fetch.subphase.ScriptFieldsContext;
|
||||
import org.elasticsearch.search.fetch.subphase.highlight.SearchHighlightContext;
|
||||
import org.elasticsearch.search.internal.ContextIndexSearcher;
|
||||
import org.elasticsearch.search.internal.ReaderContext;
|
||||
import org.elasticsearch.search.internal.ScrollContext;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
import org.elasticsearch.search.internal.SearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchContextId;
|
||||
import org.elasticsearch.search.internal.ShardSearchRequest;
|
||||
import org.elasticsearch.search.profile.Profilers;
|
||||
import org.elasticsearch.search.query.QuerySearchResult;
|
||||
|
@ -113,12 +114,18 @@ public class TestSearchContext extends SearchContext {
|
|||
}
|
||||
|
||||
public TestSearchContext(QueryShardContext queryShardContext, IndexShard indexShard, ContextIndexSearcher searcher) {
|
||||
this(queryShardContext, indexShard, searcher, null);
|
||||
}
|
||||
|
||||
public TestSearchContext(QueryShardContext queryShardContext, IndexShard indexShard,
|
||||
ContextIndexSearcher searcher, ScrollContext scrollContext) {
|
||||
this.bigArrays = null;
|
||||
this.indexService = null;
|
||||
this.fixedBitSetFilterCache = null;
|
||||
this.indexShard = indexShard;
|
||||
this.queryShardContext = queryShardContext;
|
||||
this.searcher = searcher;
|
||||
this.scrollContext = scrollContext;
|
||||
}
|
||||
|
||||
public void setSearcher(ContextIndexSearcher searcher) {
|
||||
|
@ -135,8 +142,8 @@ public class TestSearchContext extends SearchContext {
|
|||
}
|
||||
|
||||
@Override
|
||||
public SearchContextId id() {
|
||||
return new SearchContextId("", 0);
|
||||
public ShardSearchContextId id() {
|
||||
return new ShardSearchContextId("", 0);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -169,22 +176,11 @@ public class TestSearchContext extends SearchContext {
|
|||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getOriginNanoTime() {
|
||||
return originNanoTime;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ScrollContext scrollContext() {
|
||||
return scrollContext;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SearchContext scrollContext(ScrollContext scrollContext) {
|
||||
this.scrollContext = scrollContext;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SearchContextAggregations aggregations() {
|
||||
return aggregations;
|
||||
|
@ -229,10 +225,6 @@ public class TestSearchContext extends SearchContext {
|
|||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addRescore(RescoreContext rescore) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasScriptFields() {
|
||||
return false;
|
||||
|
@ -550,24 +542,6 @@ public class TestSearchContext extends SearchContext {
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void accessed(long accessTime) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public long lastAccessTime() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long keepAlive() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void keepAlive(long keepAlive) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public DfsSearchResult dfsResult() {
|
||||
return null;
|
||||
|
@ -640,4 +614,14 @@ public class TestSearchContext extends SearchContext {
|
|||
public boolean isCancelled() {
|
||||
return task.isCancelled();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addRescore(RescoreContext rescore) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReaderContext readerContext() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.elasticsearch.index.engine.EngineException;
|
|||
import org.elasticsearch.index.engine.InternalEngine;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.function.Function;
|
||||
|
||||
final class MockInternalEngine extends InternalEngine {
|
||||
private MockEngineSupport support;
|
||||
|
@ -81,4 +82,9 @@ final class MockInternalEngine extends InternalEngine {
|
|||
final Engine.Searcher engineSearcher = super.acquireSearcher(source, scope);
|
||||
return support().wrapSearcher(engineSearcher);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SearcherSupplier acquireSearcherSupplier(Function<Searcher, Searcher> wrapper, SearcherScope scope) throws EngineException {
|
||||
return super.acquireSearcherSupplier(wrapper.andThen(s -> support().wrapSearcher(s)), scope);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,59 +19,25 @@
|
|||
|
||||
package org.elasticsearch.search;
|
||||
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.OriginalIndices;
|
||||
import org.elasticsearch.action.search.SearchType;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetadata;
|
||||
import org.elasticsearch.common.lucene.search.Queries;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.util.BigArrays;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
import org.elasticsearch.index.query.QueryShardContext;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
import org.elasticsearch.search.internal.ReaderContext;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.test.TestSearchContext;
|
||||
|
||||
import static org.mockito.Mockito.mock;
|
||||
|
||||
public class MockSearchServiceTests extends ESTestCase {
|
||||
public static final IndexMetadata EMPTY_INDEX_METADATA = IndexMetadata.builder("")
|
||||
.settings(Settings.builder().put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT))
|
||||
.numberOfShards(1).numberOfReplicas(0).build();
|
||||
|
||||
public void testAssertNoInFlightContext() {
|
||||
final long nowInMillis = randomNonNegativeLong();
|
||||
SearchContext s = new TestSearchContext(new QueryShardContext(0,
|
||||
new IndexSettings(EMPTY_INDEX_METADATA, Settings.EMPTY), BigArrays.NON_RECYCLING_INSTANCE, null, null, null, null, null,
|
||||
xContentRegistry(), writableRegistry(), null, null, () -> nowInMillis, null, null, () -> true, null)) {
|
||||
|
||||
@Override
|
||||
public SearchShardTarget shardTarget() {
|
||||
return new SearchShardTarget("node", new ShardId("idx", "ignored", 0), null, OriginalIndices.NONE);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SearchType searchType() {
|
||||
return SearchType.DEFAULT;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Query query() {
|
||||
return Queries.newMatchAllQuery();
|
||||
}
|
||||
};
|
||||
MockSearchService.addActiveContext(s);
|
||||
ReaderContext reader = mock(ReaderContext.class);
|
||||
MockSearchService.addActiveContext(reader);
|
||||
try {
|
||||
Throwable e = expectThrows(AssertionError.class, () -> MockSearchService.assertNoInFlightContext());
|
||||
assertEquals("There are still [1] in-flight contexts. The first one's creation site is listed as the cause of this exception.",
|
||||
e.getMessage());
|
||||
e = e.getCause();
|
||||
// The next line with throw an exception if the date looks wrong
|
||||
assertEquals("[node][idx][0] query=[*:*]", e.getMessage());
|
||||
assertEquals(MockSearchService.class.getName(), e.getStackTrace()[0].getClassName());
|
||||
assertEquals(MockSearchServiceTests.class.getName(), e.getStackTrace()[1].getClassName());
|
||||
} finally {
|
||||
MockSearchService.removeActiveContext(s);
|
||||
MockSearchService.removeActiveContext(reader);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -11,6 +11,7 @@ import org.elasticsearch.action.ActionResponse;
|
|||
import org.elasticsearch.action.ActionType;
|
||||
import org.elasticsearch.client.node.NodeClient;
|
||||
import org.elasticsearch.common.bytes.BytesArray;
|
||||
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.concurrent.ThreadContext;
|
||||
|
@ -50,7 +51,7 @@ public class RestSubmitAsyncSearchActionTests extends ESTestCase {
|
|||
return new Task(1L, "type", "action", "description", null, null);
|
||||
}
|
||||
};
|
||||
nodeClient.initialize(new HashMap<>(), () -> "local", null);
|
||||
nodeClient.initialize(new HashMap<>(), () -> "local", null, new NamedWriteableRegistry(Collections.emptyList()));
|
||||
controller = new RestController(Collections.emptySet(), null,
|
||||
nodeClient,
|
||||
new NoneCircuitBreakerService(),
|
||||
|
|
|
@ -0,0 +1,286 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.xpack.core.search;
|
||||
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.action.admin.indices.stats.CommonStats;
|
||||
import org.elasticsearch.plugins.Plugin;
|
||||
import org.elasticsearch.xpack.core.LocalStateCompositeXPackPlugin;
|
||||
import org.elasticsearch.xpack.core.XPackClientPlugin;
|
||||
import org.elasticsearch.xpack.core.XPackSettings;
|
||||
import org.elasticsearch.xpack.core.search.action.ClosePointInTimeAction;
|
||||
import org.elasticsearch.xpack.core.search.action.ClosePointInTimeRequest;
|
||||
import org.elasticsearch.action.search.SearchPhaseExecutionException;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.SearchType;
|
||||
import org.elasticsearch.action.search.ShardSearchFailure;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetadata;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.index.IndexNotFoundException;
|
||||
import org.elasticsearch.index.IndexService;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
import org.elasticsearch.index.query.MatchAllQueryBuilder;
|
||||
import org.elasticsearch.index.query.RangeQueryBuilder;
|
||||
import org.elasticsearch.index.shard.IndexShard;
|
||||
import org.elasticsearch.indices.IndicesService;
|
||||
import org.elasticsearch.search.SearchContextMissingException;
|
||||
import org.elasticsearch.search.SearchService;
|
||||
import org.elasticsearch.test.ESIntegTestCase;
|
||||
import org.elasticsearch.xpack.core.search.action.OpenPointInTimeAction;
|
||||
import org.elasticsearch.xpack.core.search.action.OpenPointInTimeRequest;
|
||||
import org.elasticsearch.xpack.core.search.action.OpenPointInTimeResponse;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
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.arrayWithSize;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.instanceOf;
|
||||
|
||||
public class PointInTimeIT extends ESIntegTestCase {
|
||||
|
||||
@Override
|
||||
protected Settings nodeSettings(int nodeOrdinal) {
|
||||
return Settings.builder()
|
||||
.put(super.nodeSettings(nodeOrdinal))
|
||||
.put(SearchService.KEEPALIVE_INTERVAL_SETTING.getKey(), TimeValue.timeValueMillis(randomIntBetween(100, 500)))
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Collection<Class<? extends Plugin>> nodePlugins() {
|
||||
final List<Class<? extends Plugin>> plugins = new ArrayList<>();
|
||||
plugins.add(LocalStateCompositeXPackPlugin.class);
|
||||
return plugins;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Settings transportClientSettings() {
|
||||
return Settings.builder().put(super.transportClientSettings())
|
||||
.put(XPackSettings.SECURITY_ENABLED.getKey(), false).build();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Collection<Class<? extends Plugin>> transportClientPlugins() {
|
||||
final List<Class<? extends Plugin>> plugins = new ArrayList<>(super.transportClientPlugins());
|
||||
plugins.add(XPackClientPlugin.class);
|
||||
return plugins;
|
||||
}
|
||||
|
||||
public void testBasic() {
|
||||
createIndex("test");
|
||||
int numDocs = randomIntBetween(10, 50);
|
||||
for (int i = 0; i < numDocs; i++) {
|
||||
String id = Integer.toString(i);
|
||||
client().prepareIndex("test", "_doc").setId(id).setSource("value", i).get();
|
||||
}
|
||||
refresh("test");
|
||||
String readerId = openPointInTime(new String[] { "test" }, TimeValue.timeValueMinutes(2));
|
||||
SearchResponse resp1 = client().prepareSearch().setPreference(null).setSearchContext(readerId, TimeValue.timeValueMinutes(2)).get();
|
||||
assertThat(resp1.pointInTimeId(), equalTo(readerId));
|
||||
assertHitCount(resp1, numDocs);
|
||||
int deletedDocs = 0;
|
||||
for (int i = 0; i < numDocs; i++) {
|
||||
if (randomBoolean()) {
|
||||
String id = Integer.toString(i);
|
||||
client().prepareDelete("test", "_doc", id).get();
|
||||
deletedDocs++;
|
||||
}
|
||||
}
|
||||
refresh("test");
|
||||
if (randomBoolean()) {
|
||||
SearchResponse resp2 = client().prepareSearch("test").setPreference(null).setQuery(new MatchAllQueryBuilder()).get();
|
||||
assertNoFailures(resp2);
|
||||
assertHitCount(resp2, numDocs - deletedDocs);
|
||||
}
|
||||
try {
|
||||
SearchResponse resp3 = client().prepareSearch()
|
||||
.setPreference(null)
|
||||
.setQuery(new MatchAllQueryBuilder())
|
||||
.setSearchContext(resp1.pointInTimeId(), TimeValue.timeValueMinutes(2))
|
||||
.get();
|
||||
assertNoFailures(resp3);
|
||||
assertHitCount(resp3, numDocs);
|
||||
assertThat(resp3.pointInTimeId(), equalTo(readerId));
|
||||
} finally {
|
||||
closePointInTime(readerId);
|
||||
}
|
||||
}
|
||||
|
||||
public void testMultipleIndices() {
|
||||
int numIndices = randomIntBetween(1, 5);
|
||||
for (int i = 1; i <= numIndices; i++) {
|
||||
createIndex("index-" + i);
|
||||
}
|
||||
int numDocs = randomIntBetween(10, 50);
|
||||
for (int i = 0; i < numDocs; i++) {
|
||||
String id = Integer.toString(i);
|
||||
String index = "index-" + randomIntBetween(1, numIndices);
|
||||
client().prepareIndex(index, "_doc").setId(id).setSource("value", i).get();
|
||||
}
|
||||
refresh();
|
||||
String readerId = openPointInTime(new String[] { "*" }, TimeValue.timeValueMinutes(2));
|
||||
SearchResponse resp1 = client().prepareSearch().setPreference(null).setSearchContext(readerId, TimeValue.timeValueMinutes(2)).get();
|
||||
assertNoFailures(resp1);
|
||||
assertHitCount(resp1, numDocs);
|
||||
int moreDocs = randomIntBetween(10, 50);
|
||||
for (int i = 0; i < moreDocs; i++) {
|
||||
String id = "more-" + i;
|
||||
String index = "index-" + randomIntBetween(1, numIndices);
|
||||
client().prepareIndex(index, "_doc").setId(id).setSource("value", i).get();
|
||||
}
|
||||
refresh();
|
||||
try {
|
||||
SearchResponse resp2 = client().prepareSearch().get();
|
||||
assertNoFailures(resp2);
|
||||
assertHitCount(resp2, numDocs + moreDocs);
|
||||
|
||||
SearchResponse resp3 = client().prepareSearch()
|
||||
.setPreference(null)
|
||||
.setSearchContext(resp1.pointInTimeId(), TimeValue.timeValueMinutes(1))
|
||||
.get();
|
||||
assertNoFailures(resp3);
|
||||
assertHitCount(resp3, numDocs);
|
||||
} finally {
|
||||
closePointInTime(resp1.pointInTimeId());
|
||||
}
|
||||
}
|
||||
|
||||
public void testPointInTimeNotFound() throws Exception {
|
||||
createIndex("index");
|
||||
int index1 = randomIntBetween(10, 50);
|
||||
for (int i = 0; i < index1; i++) {
|
||||
String id = Integer.toString(i);
|
||||
client().prepareIndex("index", "_doc").setId(id).setSource("value", i).get();
|
||||
}
|
||||
refresh();
|
||||
String readerId = openPointInTime(new String[] { "index" }, TimeValue.timeValueSeconds(5));
|
||||
SearchResponse resp1 = client().prepareSearch()
|
||||
.setPreference(null)
|
||||
.setSearchContext(readerId, TimeValue.timeValueMillis(randomIntBetween(0, 10)))
|
||||
.get();
|
||||
assertNoFailures(resp1);
|
||||
assertHitCount(resp1, index1);
|
||||
if (rarely()) {
|
||||
assertBusy(() -> {
|
||||
final CommonStats stats = client().admin().indices().prepareStats().setSearch(true).get().getTotal();
|
||||
assertThat(stats.search.getOpenContexts(), equalTo(0L));
|
||||
}, 60, TimeUnit.SECONDS);
|
||||
} else {
|
||||
closePointInTime(resp1.pointInTimeId());
|
||||
}
|
||||
SearchPhaseExecutionException e = expectThrows(
|
||||
SearchPhaseExecutionException.class,
|
||||
() -> client().prepareSearch()
|
||||
.setPreference(null)
|
||||
.setSearchContext(resp1.pointInTimeId(), TimeValue.timeValueMinutes(1))
|
||||
.get()
|
||||
);
|
||||
for (ShardSearchFailure failure : e.shardFailures()) {
|
||||
assertThat(ExceptionsHelper.unwrapCause(failure.getCause()), instanceOf(SearchContextMissingException.class));
|
||||
}
|
||||
}
|
||||
|
||||
public void testIndexNotFound() {
|
||||
createIndex("index-1");
|
||||
createIndex("index-2");
|
||||
|
||||
int index1 = randomIntBetween(10, 50);
|
||||
for (int i = 0; i < index1; i++) {
|
||||
String id = Integer.toString(i);
|
||||
client().prepareIndex("index-1", "_doc").setId(id).setSource("value", i).get();
|
||||
}
|
||||
|
||||
int index2 = randomIntBetween(10, 50);
|
||||
for (int i = 0; i < index2; i++) {
|
||||
String id = Integer.toString(i);
|
||||
client().prepareIndex("index-2", "_doc").setId(id).setSource("value", i).get();
|
||||
}
|
||||
refresh();
|
||||
String readerId = openPointInTime(new String[] { "index-*" }, TimeValue.timeValueMinutes(2));
|
||||
SearchResponse resp1 = client().prepareSearch().setPreference(null).setSearchContext(readerId, TimeValue.timeValueMinutes(2)).get();
|
||||
assertNoFailures(resp1);
|
||||
assertHitCount(resp1, index1 + index2);
|
||||
client().admin().indices().prepareDelete("index-1").get();
|
||||
if (randomBoolean()) {
|
||||
SearchResponse resp2 = client().prepareSearch("index-*").get();
|
||||
assertNoFailures(resp2);
|
||||
assertHitCount(resp2, index2);
|
||||
|
||||
}
|
||||
expectThrows(
|
||||
IndexNotFoundException.class,
|
||||
() -> client().prepareSearch()
|
||||
.setPreference(null)
|
||||
.setSearchContext(resp1.pointInTimeId(), TimeValue.timeValueMinutes(1))
|
||||
.get()
|
||||
);
|
||||
closePointInTime(resp1.pointInTimeId());
|
||||
}
|
||||
|
||||
public void testCanMatch() throws Exception {
|
||||
final Settings.Builder settings = Settings.builder()
|
||||
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, randomIntBetween(5, 10))
|
||||
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0)
|
||||
.put(IndexSettings.INDEX_SEARCH_IDLE_AFTER.getKey(), TimeValue.timeValueMillis(randomIntBetween(50, 100)));
|
||||
assertAcked(
|
||||
prepareCreate("test").setSettings(settings).addMapping("_doc", "created_date", "type=date,format=yyyy-MM-dd"));
|
||||
ensureGreen("test");
|
||||
String readerId = openPointInTime(new String[] { "test*" }, TimeValue.timeValueMinutes(2));
|
||||
try {
|
||||
for (String node : internalCluster().nodesInclude("test")) {
|
||||
for (IndexService indexService : internalCluster().getInstance(IndicesService.class, node)) {
|
||||
for (IndexShard indexShard : indexService) {
|
||||
assertBusy(() -> assertTrue(indexShard.isSearchIdle()));
|
||||
}
|
||||
}
|
||||
}
|
||||
client().prepareIndex("test", "_doc").setId("1").setSource("created_date", "2020-01-01").get();
|
||||
SearchResponse resp = client().prepareSearch()
|
||||
.setQuery(new RangeQueryBuilder("created_date").gte("2020-01-02").lte("2020-01-03"))
|
||||
.setSearchType(SearchType.QUERY_THEN_FETCH)
|
||||
.setPreference(null)
|
||||
.setPreFilterShardSize(randomIntBetween(2, 3))
|
||||
.setMaxConcurrentShardRequests(randomIntBetween(1, 2))
|
||||
.setSearchContext(readerId, TimeValue.timeValueMinutes(2))
|
||||
.get();
|
||||
assertThat(resp.getHits().getHits(), arrayWithSize(0));
|
||||
for (String node : internalCluster().nodesInclude("test")) {
|
||||
for (IndexService indexService : internalCluster().getInstance(IndicesService.class, node)) {
|
||||
for (IndexShard indexShard : indexService) {
|
||||
// all shards are still search-idle as we did not acquire new searchers
|
||||
assertTrue(indexShard.isSearchIdle());
|
||||
}
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
closePointInTime(readerId);
|
||||
}
|
||||
}
|
||||
|
||||
private String openPointInTime(String[] indices, TimeValue keepAlive) {
|
||||
OpenPointInTimeRequest request = new OpenPointInTimeRequest(
|
||||
indices,
|
||||
OpenPointInTimeRequest.DEFAULT_INDICES_OPTIONS,
|
||||
keepAlive,
|
||||
null,
|
||||
null
|
||||
);
|
||||
final OpenPointInTimeResponse response = client().execute(OpenPointInTimeAction.INSTANCE, request).actionGet();
|
||||
return response.getSearchContextId();
|
||||
}
|
||||
|
||||
private void closePointInTime(String readerId) {
|
||||
client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(readerId)).actionGet();
|
||||
}
|
||||
}
|
|
@ -192,7 +192,9 @@ import org.elasticsearch.xpack.core.rollup.action.StartRollupJobAction;
|
|||
import org.elasticsearch.xpack.core.rollup.action.StopRollupJobAction;
|
||||
import org.elasticsearch.xpack.core.rollup.job.RollupJob;
|
||||
import org.elasticsearch.xpack.core.rollup.job.RollupJobStatus;
|
||||
import org.elasticsearch.xpack.core.search.action.ClosePointInTimeAction;
|
||||
import org.elasticsearch.xpack.core.search.action.GetAsyncSearchAction;
|
||||
import org.elasticsearch.xpack.core.search.action.OpenPointInTimeAction;
|
||||
import org.elasticsearch.xpack.core.search.action.SubmitAsyncSearchAction;
|
||||
import org.elasticsearch.xpack.core.searchablesnapshots.SearchableSnapshotFeatureSetUsage;
|
||||
import org.elasticsearch.xpack.core.security.SecurityFeatureSetUsage;
|
||||
|
@ -485,7 +487,10 @@ public class XPackClientPlugin extends Plugin implements ActionPlugin, NetworkPl
|
|||
// Async Search
|
||||
SubmitAsyncSearchAction.INSTANCE,
|
||||
GetAsyncSearchAction.INSTANCE,
|
||||
DeleteAsyncResultAction.INSTANCE
|
||||
DeleteAsyncResultAction.INSTANCE,
|
||||
// Point in time
|
||||
OpenPointInTimeAction.INSTANCE,
|
||||
ClosePointInTimeAction.INSTANCE
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -71,6 +71,12 @@ import org.elasticsearch.xpack.core.ml.MlMetadata;
|
|||
import org.elasticsearch.xpack.core.rest.action.RestReloadAnalyzersAction;
|
||||
import org.elasticsearch.xpack.core.rest.action.RestXPackInfoAction;
|
||||
import org.elasticsearch.xpack.core.rest.action.RestXPackUsageAction;
|
||||
import org.elasticsearch.xpack.core.search.action.ClosePointInTimeAction;
|
||||
import org.elasticsearch.xpack.core.search.action.OpenPointInTimeAction;
|
||||
import org.elasticsearch.xpack.core.search.action.RestClosePointInTimeAction;
|
||||
import org.elasticsearch.xpack.core.search.action.RestOpenPointInTimeAction;
|
||||
import org.elasticsearch.xpack.core.search.action.TransportClosePointInTimeAction;
|
||||
import org.elasticsearch.xpack.core.search.action.TransportOpenPointInTimeAction;
|
||||
import org.elasticsearch.xpack.core.security.authc.TokenMetadata;
|
||||
import org.elasticsearch.xpack.core.ssl.SSLConfiguration;
|
||||
import org.elasticsearch.xpack.core.ssl.SSLConfigurationReloader;
|
||||
|
@ -297,6 +303,8 @@ public class XPackPlugin extends XPackClientPlugin implements ExtensiblePlugin,
|
|||
actions.addAll(licensing.getActions());
|
||||
actions.add(new ActionHandler<>(ReloadAnalyzerAction.INSTANCE, TransportReloadAnalyzersAction.class));
|
||||
actions.add(new ActionHandler<>(DeleteAsyncResultAction.INSTANCE, TransportDeleteAsyncResultAction.class));
|
||||
actions.add(new ActionHandler<>(OpenPointInTimeAction.INSTANCE, TransportOpenPointInTimeAction.class));
|
||||
actions.add(new ActionHandler<>(ClosePointInTimeAction.INSTANCE, TransportClosePointInTimeAction.class));
|
||||
return actions;
|
||||
}
|
||||
|
||||
|
@ -325,6 +333,8 @@ public class XPackPlugin extends XPackClientPlugin implements ExtensiblePlugin,
|
|||
handlers.add(new RestReloadAnalyzersAction());
|
||||
handlers.addAll(licensing.getRestHandlers(settings, restController, clusterSettings, indexScopedSettings, settingsFilter,
|
||||
indexNameExpressionResolver, nodesInCluster));
|
||||
handlers.add(new RestOpenPointInTimeAction());
|
||||
handlers.add(new RestClosePointInTimeAction());
|
||||
return handlers;
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,19 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.xpack.core.search.action;
|
||||
|
||||
import org.elasticsearch.action.ActionType;
|
||||
|
||||
public class ClosePointInTimeAction extends ActionType<ClosePointInTimeResponse> {
|
||||
|
||||
public static final ClosePointInTimeAction INSTANCE = new ClosePointInTimeAction();
|
||||
public static final String NAME = "indices:data/read/close_point_in_time";
|
||||
|
||||
private ClosePointInTimeAction() {
|
||||
super(NAME, ClosePointInTimeResponse::new);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,85 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.xpack.core.search.action;
|
||||
|
||||
import org.elasticsearch.action.ActionRequest;
|
||||
import org.elasticsearch.action.ActionRequestValidationException;
|
||||
import org.elasticsearch.common.ParseField;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.ToXContentObject;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class ClosePointInTimeRequest extends ActionRequest implements ToXContentObject {
|
||||
private static final ParseField ID = new ParseField("id");
|
||||
|
||||
private final String id;
|
||||
|
||||
public ClosePointInTimeRequest(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
this.id = in.readString();
|
||||
}
|
||||
|
||||
public ClosePointInTimeRequest(String id) {
|
||||
this.id = id;
|
||||
}
|
||||
|
||||
public String getId() {
|
||||
return id;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ActionRequestValidationException validate() {
|
||||
if (Strings.isEmpty(id)) {
|
||||
throw new IllegalArgumentException("reader id must be specified");
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeString(id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
builder.field(ID.getPreferredName(), id);
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
|
||||
public static ClosePointInTimeRequest fromXContent(XContentParser parser) throws IOException {
|
||||
if (parser.nextToken() != XContentParser.Token.START_OBJECT) {
|
||||
throw new IllegalArgumentException("Malformed content, must start with an object");
|
||||
} else {
|
||||
XContentParser.Token token;
|
||||
String id = null;
|
||||
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
|
||||
if (token == XContentParser.Token.FIELD_NAME && parser.currentName().equals(ID.getPreferredName())) {
|
||||
token = parser.nextToken();
|
||||
if (token.isValue() == false) {
|
||||
throw new IllegalArgumentException("the request must contain only [" + ID.getPreferredName() + " field");
|
||||
}
|
||||
id = parser.text();
|
||||
} else {
|
||||
throw new IllegalArgumentException("Unknown parameter [" + parser.currentName() +
|
||||
"] in request body or parameter is of the wrong type[" + token + "] ");
|
||||
}
|
||||
}
|
||||
if (Strings.isNullOrEmpty(id)) {
|
||||
throw new IllegalArgumentException("search context id is is not provided");
|
||||
}
|
||||
return new ClosePointInTimeRequest(id);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,22 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.xpack.core.search.action;
|
||||
|
||||
import org.elasticsearch.action.search.ClearScrollResponse;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class ClosePointInTimeResponse extends ClearScrollResponse {
|
||||
public ClosePointInTimeResponse(boolean succeeded, int numFreed) {
|
||||
super(succeeded, numFreed);
|
||||
}
|
||||
|
||||
public ClosePointInTimeResponse(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
}
|
||||
}
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue