This commit ensures that we rewrite the shard request with a short-lived can_match searcher. This is required for frozen indices since the high level rewrite is now performed on a network thread where we don't want to perform I/O. Closes #53985
This commit is contained in:
parent
381d7586e4
commit
3b4751bdb7
|
@ -1228,12 +1228,12 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Acquire the searcher without applying the additional reader wrapper.
|
* Acquire a lightweight searcher which can be used to rewrite shard search requests.
|
||||||
*/
|
*/
|
||||||
public Engine.Searcher acquireSearcherNoWrap(String source) {
|
public Engine.Searcher acquireCanMatchSearcher() {
|
||||||
readAllowed();
|
readAllowed();
|
||||||
markSearcherAccessed();
|
markSearcherAccessed();
|
||||||
return getEngine().acquireSearcher(source, Engine.SearcherScope.EXTERNAL);
|
return getEngine().acquireSearcher("can_match", Engine.SearcherScope.EXTERNAL);
|
||||||
}
|
}
|
||||||
|
|
||||||
public Engine.Searcher acquireSearcher(String source) {
|
public Engine.Searcher acquireSearcher(String source) {
|
||||||
|
@ -1252,10 +1252,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||||
return wrapSearcher(searcher);
|
return wrapSearcher(searcher);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
private Engine.Searcher wrapSearcher(Engine.Searcher searcher) {
|
||||||
* Wraps the provided searcher acquired with {@link #acquireSearcherNoWrap(String)}.
|
|
||||||
*/
|
|
||||||
public Engine.Searcher wrapSearcher(Engine.Searcher searcher) {
|
|
||||||
assert ElasticsearchDirectoryReader.unwrap(searcher.getDirectoryReader())
|
assert ElasticsearchDirectoryReader.unwrap(searcher.getDirectoryReader())
|
||||||
!= null : "DirectoryReader must be an instance or ElasticsearchDirectoryReader";
|
!= null : "DirectoryReader must be an instance or ElasticsearchDirectoryReader";
|
||||||
boolean success = false;
|
boolean success = false;
|
||||||
|
|
|
@ -33,6 +33,7 @@ import org.elasticsearch.action.search.SearchShardTask;
|
||||||
import org.elasticsearch.action.search.SearchType;
|
import org.elasticsearch.action.search.SearchType;
|
||||||
import org.elasticsearch.cluster.ClusterState;
|
import org.elasticsearch.cluster.ClusterState;
|
||||||
import org.elasticsearch.cluster.service.ClusterService;
|
import org.elasticsearch.cluster.service.ClusterService;
|
||||||
|
import org.elasticsearch.common.CheckedSupplier;
|
||||||
import org.elasticsearch.common.UUIDs;
|
import org.elasticsearch.common.UUIDs;
|
||||||
import org.elasticsearch.common.breaker.CircuitBreaker;
|
import org.elasticsearch.common.breaker.CircuitBreaker;
|
||||||
import org.elasticsearch.common.component.AbstractLifecycleComponent;
|
import org.elasticsearch.common.component.AbstractLifecycleComponent;
|
||||||
|
@ -315,31 +316,24 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
|
||||||
}
|
}
|
||||||
|
|
||||||
public void executeDfsPhase(ShardSearchRequest request, SearchShardTask task, ActionListener<SearchPhaseResult> listener) {
|
public void executeDfsPhase(ShardSearchRequest request, SearchShardTask task, ActionListener<SearchPhaseResult> listener) {
|
||||||
rewriteShardRequest(request, ActionListener.wrap(
|
IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex());
|
||||||
// fork the execution in the search thread pool and wraps the searcher
|
IndexShard shard = indexService.getShard(request.shardId().id());
|
||||||
// to execute the query
|
rewriteAndFetchShardRequest(shard, request, new ActionListener<ShardSearchRequest>() {
|
||||||
context -> {
|
@Override
|
||||||
try {
|
public void onResponse(ShardSearchRequest rewritten) {
|
||||||
context.wrapSearcher().execute(() -> {
|
// fork the execution in the search thread pool
|
||||||
final SearchPhaseResult result;
|
runAsync(shard, () -> executeDfsPhase(request, task), listener);
|
||||||
try {
|
|
||||||
result = executeDfsPhase(context, task);
|
|
||||||
} catch (Exception exc) {
|
|
||||||
listener.onFailure(exc);
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
listener.onResponse(result);
|
|
||||||
});
|
|
||||||
} catch (Exception exc) {
|
|
||||||
// if the execution is rejected we need to close the searcher
|
|
||||||
IOUtils.closeWhileHandlingException(context.searcher);
|
|
||||||
listener.onFailure(exc);
|
|
||||||
}
|
|
||||||
}, listener::onFailure));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private DfsSearchResult executeDfsPhase(SearchRewriteContext rewriteContext, SearchShardTask task) throws IOException {
|
@Override
|
||||||
final SearchContext context = createAndPutContext(rewriteContext, task);
|
public void onFailure(Exception exc) {
|
||||||
|
listener.onFailure(exc);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
private DfsSearchResult executeDfsPhase(ShardSearchRequest request, SearchShardTask task) throws IOException {
|
||||||
|
final SearchContext context = createAndPutContext(request, task);
|
||||||
context.incRef();
|
context.incRef();
|
||||||
try {
|
try {
|
||||||
contextProcessing(context);
|
contextProcessing(context);
|
||||||
|
@ -371,58 +365,66 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
|
||||||
public void executeQueryPhase(ShardSearchRequest request, SearchShardTask task, ActionListener<SearchPhaseResult> listener) {
|
public void executeQueryPhase(ShardSearchRequest request, SearchShardTask task, ActionListener<SearchPhaseResult> listener) {
|
||||||
assert request.canReturnNullResponseIfMatchNoDocs() == false || request.numberOfShards() > 1
|
assert request.canReturnNullResponseIfMatchNoDocs() == false || request.numberOfShards() > 1
|
||||||
: "empty responses require more than one shard";
|
: "empty responses require more than one shard";
|
||||||
rewriteShardRequest(request, ActionListener.wrap(
|
IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex());
|
||||||
context -> {
|
IndexShard shard = indexService.getShard(request.shardId().id());
|
||||||
|
rewriteAndFetchShardRequest(shard, request, new ActionListener<ShardSearchRequest>() {
|
||||||
|
@Override
|
||||||
|
public void onResponse(ShardSearchRequest orig) {
|
||||||
|
if (orig.canReturnNullResponseIfMatchNoDocs()) {
|
||||||
|
// we clone the shard request and perform a quick rewrite using a lightweight
|
||||||
|
// searcher since we are outside of the search thread pool.
|
||||||
|
// If the request rewrites to "match none" we can shortcut the query phase
|
||||||
|
// entirely. Otherwise we fork the execution in the search thread pool.
|
||||||
|
ShardSearchRequest canMatchRequest = new ShardSearchRequest(orig);
|
||||||
|
try (Engine.Searcher searcher = shard.acquireCanMatchSearcher()) {
|
||||||
|
QueryShardContext context = indexService.newQueryShardContext(request.shardId().id(), searcher,
|
||||||
|
request::nowInMillis, request.getClusterAlias());
|
||||||
|
Rewriteable.rewrite(request.getRewriteable(), context, true);
|
||||||
|
} catch (Exception exc) {
|
||||||
|
listener.onFailure(exc);
|
||||||
|
}
|
||||||
|
if (canRewriteToMatchNone(canMatchRequest.source())
|
||||||
|
&& canMatchRequest.source().query() instanceof MatchNoneQueryBuilder) {
|
||||||
|
assert canMatchRequest.scroll() == null : "must always create search context for scroll requests";
|
||||||
|
listener.onResponse(QuerySearchResult.nullInstance());
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// fork the execution in the search thread pool
|
||||||
|
runAsync(shard, () -> executeQueryPhase(orig, task), listener);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onFailure(Exception exc) {
|
||||||
|
listener.onFailure(exc);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
private <T> void runAsync(IndexShard shard, CheckedSupplier<T, Exception> command, ActionListener<T> listener) {
|
||||||
|
Executor executor = getExecutor(shard);
|
||||||
try {
|
try {
|
||||||
ShardSearchRequest rewritten = context.request;
|
executor.execute(() -> {
|
||||||
if (rewritten.canReturnNullResponseIfMatchNoDocs()
|
T result;
|
||||||
&& canRewriteToMatchNone(rewritten.source())
|
|
||||||
&& rewritten.source().query() instanceof MatchNoneQueryBuilder) {
|
|
||||||
assert request.scroll() == null : "must always create search context for scroll requests";
|
|
||||||
onMatchNoDocs(context, listener);
|
|
||||||
} else {
|
|
||||||
// fork the execution in the search thread pool and wraps the searcher
|
|
||||||
// to execute the query
|
|
||||||
context.wrapSearcher().execute(() -> {
|
|
||||||
final SearchPhaseResult result;
|
|
||||||
try {
|
try {
|
||||||
result = executeQueryPhase(context, task);
|
result = command.get();
|
||||||
} catch (Exception exc) {
|
} catch (Exception exc) {
|
||||||
listener.onFailure(exc);
|
listener.onFailure(exc);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
listener.onResponse(result);
|
listener.onResponse(result);
|
||||||
});
|
});
|
||||||
}
|
|
||||||
} catch (Exception exc) {
|
|
||||||
// if the execution is rejected we need to close the searcher
|
|
||||||
IOUtils.closeWhileHandlingException(context.searcher);
|
|
||||||
listener.onFailure(exc);
|
|
||||||
}
|
|
||||||
}, listener::onFailure));
|
|
||||||
}
|
|
||||||
|
|
||||||
private void onMatchNoDocs(SearchRewriteContext rewriteContext, ActionListener<SearchPhaseResult> listener) throws IOException {
|
|
||||||
// creates a lightweight search context that we use to inform context listeners
|
|
||||||
// before closing
|
|
||||||
SearchContext searchContext = createSearchContext(rewriteContext, defaultSearchTimeout);
|
|
||||||
try (SearchContext dummy = searchContext) {
|
|
||||||
onNewContext(searchContext);
|
|
||||||
onFreeContext(searchContext);
|
|
||||||
} catch (Exception exc) {
|
} catch (Exception exc) {
|
||||||
listener.onFailure(exc);
|
listener.onFailure(exc);
|
||||||
return;
|
|
||||||
}
|
}
|
||||||
listener.onResponse(QuerySearchResult.nullInstance());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private <T> void runAsync(SearchContextId contextId, Supplier<T> executable, ActionListener<T> listener) {
|
private <T> void runAsync(SearchContextId contextId, Supplier<T> executable, ActionListener<T> listener) {
|
||||||
getExecutor(contextId).execute(ActionRunnable.supply(listener, executable::get));
|
getExecutor(contextId).execute(ActionRunnable.supply(listener, executable::get));
|
||||||
}
|
}
|
||||||
|
|
||||||
private SearchPhaseResult executeQueryPhase(SearchRewriteContext rewriteContext, SearchShardTask task) throws Exception {
|
private SearchPhaseResult executeQueryPhase(ShardSearchRequest request, SearchShardTask task) throws Exception {
|
||||||
final SearchContext context = createAndPutContext(rewriteContext, task);
|
final SearchContext context = createAndPutContext(request, task);
|
||||||
final ShardSearchRequest request = rewriteContext.request;
|
|
||||||
context.incRef();
|
context.incRef();
|
||||||
try {
|
try {
|
||||||
final long afterQueryTime;
|
final long afterQueryTime;
|
||||||
|
@ -624,8 +626,8 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
final SearchContext createAndPutContext(SearchRewriteContext rewriteContext, SearchShardTask task) throws IOException {
|
final SearchContext createAndPutContext(ShardSearchRequest request, SearchShardTask task) throws IOException {
|
||||||
SearchContext context = createContext(rewriteContext, task);
|
SearchContext context = createContext(request, task);
|
||||||
onNewContext(context);
|
onNewContext(context);
|
||||||
boolean success = false;
|
boolean success = false;
|
||||||
try {
|
try {
|
||||||
|
@ -658,10 +660,9 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
final SearchContext createContext(SearchRewriteContext rewriteContext, SearchShardTask searchTask) throws IOException {
|
final SearchContext createContext(ShardSearchRequest request, SearchShardTask searchTask) throws IOException {
|
||||||
final DefaultSearchContext context = createSearchContext(rewriteContext, defaultSearchTimeout);
|
final DefaultSearchContext context = createSearchContext(request, defaultSearchTimeout);
|
||||||
try {
|
try {
|
||||||
final ShardSearchRequest request = rewriteContext.request;
|
|
||||||
if (request.scroll() != null) {
|
if (request.scroll() != null) {
|
||||||
context.addReleasable(openScrollContexts::decrementAndGet, Lifetime.CONTEXT);
|
context.addReleasable(openScrollContexts::decrementAndGet, Lifetime.CONTEXT);
|
||||||
if (openScrollContexts.incrementAndGet() > maxOpenScrollContext) {
|
if (openScrollContexts.incrementAndGet() > maxOpenScrollContext) {
|
||||||
|
@ -704,21 +705,19 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
|
||||||
}
|
}
|
||||||
|
|
||||||
public DefaultSearchContext createSearchContext(ShardSearchRequest request, TimeValue timeout) throws IOException {
|
public DefaultSearchContext createSearchContext(ShardSearchRequest request, TimeValue timeout) throws IOException {
|
||||||
IndexShard shard = indicesService.indexServiceSafe(request.shardId().getIndex()).getShard(request.shardId().id());
|
return createSearchContext(request, timeout, "search");
|
||||||
SearchRewriteContext rewriteContext = acquireSearcherAndRewrite(request, shard);
|
|
||||||
// make sure that we wrap the searcher when executing the query
|
|
||||||
return createSearchContext(rewriteContext.wrapSearcher(), timeout);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private DefaultSearchContext createSearchContext(SearchRewriteContext rewriteContext, TimeValue timeout) throws IOException {
|
private DefaultSearchContext createSearchContext(ShardSearchRequest request, TimeValue timeout, String source) throws IOException {
|
||||||
boolean success = false;
|
|
||||||
try {
|
|
||||||
final ShardSearchRequest request = rewriteContext.request;
|
|
||||||
final Engine.Searcher searcher = rewriteContext.searcher;
|
|
||||||
IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex());
|
IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex());
|
||||||
IndexShard indexShard = indexService.getShard(request.shardId().getId());
|
IndexShard indexShard = indexService.getShard(request.shardId().getId());
|
||||||
SearchShardTarget shardTarget = new SearchShardTarget(clusterService.localNode().getId(),
|
SearchShardTarget shardTarget = new SearchShardTarget(clusterService.localNode().getId(),
|
||||||
indexShard.shardId(), request.getClusterAlias(), OriginalIndices.NONE);
|
indexShard.shardId(), request.getClusterAlias(), OriginalIndices.NONE);
|
||||||
|
Engine.Searcher searcher = indexShard.acquireSearcher(source);
|
||||||
|
|
||||||
|
boolean success = false;
|
||||||
|
DefaultSearchContext searchContext = null;
|
||||||
|
try {
|
||||||
// TODO: If no changes are made since the last commit, and the searcher is opened from that commit, then we can use the
|
// TODO: If no changes are made since the last commit, and the searcher is opened from that commit, then we can use the
|
||||||
// commit_id as the context_id. And if the local checkpoint and max_seq_no of that commit equal the global checkpoint,
|
// commit_id as the context_id. And if the local checkpoint and max_seq_no of that commit equal the global checkpoint,
|
||||||
// then we can use a combination of history_uuid and one of these values as a **weaker** context_id.
|
// then we can use a combination of history_uuid and one of these values as a **weaker** context_id.
|
||||||
|
@ -727,19 +726,24 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
|
||||||
// the Lucene doc ids can be different.
|
// the Lucene doc ids can be different.
|
||||||
final String readerId = UUIDs.base64UUID();
|
final String readerId = UUIDs.base64UUID();
|
||||||
final SearchContextId searchContextId = new SearchContextId(readerId, idGenerator.incrementAndGet());
|
final SearchContextId searchContextId = new SearchContextId(readerId, idGenerator.incrementAndGet());
|
||||||
DefaultSearchContext searchContext = new DefaultSearchContext(searchContextId, request, shardTarget,
|
searchContext = new DefaultSearchContext(searchContextId, request, shardTarget,
|
||||||
searcher, clusterService, indexService, indexShard, bigArrays, threadPool::relativeTimeInMillis, timeout,
|
searcher, clusterService, indexService, indexShard, bigArrays, threadPool::relativeTimeInMillis, timeout,
|
||||||
fetchPhase, lowLevelCancellation, clusterService.state().nodes().getMinNodeVersion());
|
fetchPhase, lowLevelCancellation, clusterService.state().nodes().getMinNodeVersion());
|
||||||
|
// we clone the query shard context here just for rewriting otherwise we
|
||||||
|
// might end up with incorrect state since we are using now() or script services
|
||||||
|
// during rewrite and normalized / evaluate templates etc.
|
||||||
|
QueryShardContext context = new QueryShardContext(searchContext.getQueryShardContext());
|
||||||
|
Rewriteable.rewrite(request.getRewriteable(), context, true);
|
||||||
|
assert searchContext.getQueryShardContext().isCacheable();
|
||||||
success = true;
|
success = true;
|
||||||
return searchContext;
|
|
||||||
} finally {
|
} finally {
|
||||||
if (success == false) {
|
if (success == false) {
|
||||||
// we handle the case where `IndicesService#indexServiceSafe`or `IndexService#getShard`, or the DefaultSearchContext
|
// we handle the case where the DefaultSearchContext constructor throws an exception since we would otherwise
|
||||||
// constructor throws an exception since we would otherwise leak a searcher and this can have severe implications
|
// leak a searcher and this can have severe implications (unable to obtain shard lock exceptions).
|
||||||
// (unable to obtain shard lock exceptions).
|
IOUtils.closeWhileHandlingException(searcher);
|
||||||
IOUtils.closeWhileHandlingException(rewriteContext.searcher);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
return searchContext;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -1110,7 +1114,7 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
|
||||||
IndexShard indexShard = indexService.getShard(request.shardId().getId());
|
IndexShard indexShard = indexService.getShard(request.shardId().getId());
|
||||||
// we don't want to use the reader wrapper since it could run costly operations
|
// we don't want to use the reader wrapper since it could run costly operations
|
||||||
// and we can afford false positives.
|
// and we can afford false positives.
|
||||||
try (Engine.Searcher searcher = indexShard.acquireSearcherNoWrap("can_match")) {
|
try (Engine.Searcher searcher = indexShard.acquireCanMatchSearcher()) {
|
||||||
QueryShardContext context = indexService.newQueryShardContext(request.shardId().id(), searcher,
|
QueryShardContext context = indexService.newQueryShardContext(request.shardId().id(), searcher,
|
||||||
request::nowInMillis, request.getClusterAlias());
|
request::nowInMillis, request.getClusterAlias());
|
||||||
Rewriteable.rewrite(request.getRewriteable(), context, false);
|
Rewriteable.rewrite(request.getRewriteable(), context, false);
|
||||||
|
@ -1146,50 +1150,17 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
|
||||||
return aggregations == null || aggregations.mustVisitAllDocs() == false;
|
return aggregations == null || aggregations.mustVisitAllDocs() == false;
|
||||||
}
|
}
|
||||||
|
|
||||||
/*
|
private void rewriteAndFetchShardRequest(IndexShard shard, ShardSearchRequest request, ActionListener<ShardSearchRequest> listener) {
|
||||||
* Rewrites the search request with a light weight rewrite context in order to fetch resources asynchronously
|
|
||||||
* and then rewrites with a searcher when the shard is active.
|
|
||||||
* The provided action listener is executed on the same thread or in a listener threadpool.
|
|
||||||
*/
|
|
||||||
private void rewriteShardRequest(ShardSearchRequest request, ActionListener<SearchRewriteContext> listener) {
|
|
||||||
IndexShard shard = indicesService.indexServiceSafe(request.shardId().getIndex()).getShard(request.shardId().id());
|
|
||||||
ActionListener<Rewriteable> actionListener = ActionListener.wrap(r ->
|
ActionListener<Rewriteable> actionListener = ActionListener.wrap(r ->
|
||||||
// now we need to check if there is a pending refresh and register
|
// now we need to check if there is a pending refresh and register
|
||||||
shard.awaitShardSearchActive(b -> {
|
shard.awaitShardSearchActive(b -> listener.onResponse(request)),
|
||||||
try {
|
listener::onFailure);
|
||||||
// we can now acquire a searcher and rewrite the request with it
|
|
||||||
SearchRewriteContext rewriteContext = acquireSearcherAndRewrite(request, shard);
|
|
||||||
listener.onResponse(rewriteContext);
|
|
||||||
} catch (Exception e) {
|
|
||||||
listener.onFailure(e);
|
|
||||||
}
|
|
||||||
}), listener::onFailure);
|
|
||||||
// we also do rewrite on the coordinating node (TransportSearchService) but we also need to do it here for BWC as well as
|
// we also do rewrite on the coordinating node (TransportSearchService) but we also need to do it here for BWC as well as
|
||||||
// AliasFilters that might need to be rewritten. These are edge-cases but we are every efficient doing the rewrite here so it's not
|
// AliasFilters that might need to be rewritten. These are edge-cases but we are every efficient doing the rewrite here so it's not
|
||||||
// adding a lot of overhead
|
// adding a lot of overhead
|
||||||
Rewriteable.rewriteAndFetch(request.getRewriteable(), indicesService.getRewriteContext(request::nowInMillis), actionListener);
|
Rewriteable.rewriteAndFetch(request.getRewriteable(), indicesService.getRewriteContext(request::nowInMillis), actionListener);
|
||||||
}
|
}
|
||||||
|
|
||||||
SearchRewriteContext acquireSearcherAndRewrite(ShardSearchRequest request, IndexShard shard) throws IOException {
|
|
||||||
// acquire the searcher for rewrite with no wrapping in order to avoid costly
|
|
||||||
// operations. We'll wrap the searcher at a later stage (when executing the query).
|
|
||||||
Engine.Searcher searcher = shard.acquireSearcherNoWrap("search");
|
|
||||||
boolean success = false;
|
|
||||||
try {
|
|
||||||
IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex());
|
|
||||||
QueryShardContext context = indexService.newQueryShardContext(request.shardId().id(), searcher,
|
|
||||||
request::nowInMillis, request.getClusterAlias());
|
|
||||||
Rewriteable.rewrite(request.getRewriteable(), context, true);
|
|
||||||
SearchRewriteContext rewrite = new SearchRewriteContext(request, shard, searcher, getExecutor(shard));
|
|
||||||
success = true;
|
|
||||||
return rewrite;
|
|
||||||
} finally {
|
|
||||||
if (success == false) {
|
|
||||||
IOUtils.closeWhileHandlingException(searcher);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns a new {@link QueryRewriteContext} with the given {@code now} provider
|
* Returns a new {@link QueryRewriteContext} with the given {@code now} provider
|
||||||
*/
|
*/
|
||||||
|
@ -1228,37 +1199,6 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
|
||||||
return request.source().aggregations().buildPipelineTree();
|
return request.source().aggregations().buildPipelineTree();
|
||||||
}
|
}
|
||||||
|
|
||||||
static class SearchRewriteContext {
|
|
||||||
private final ShardSearchRequest request;
|
|
||||||
private final IndexShard shard;
|
|
||||||
private Engine.Searcher searcher;
|
|
||||||
private final Executor executor;
|
|
||||||
|
|
||||||
private boolean isWrapped;
|
|
||||||
|
|
||||||
private SearchRewriteContext(ShardSearchRequest request,
|
|
||||||
IndexShard shard,
|
|
||||||
Engine.Searcher searcher,
|
|
||||||
Executor executor) {
|
|
||||||
this.request = request;
|
|
||||||
this.shard = shard;
|
|
||||||
this.searcher = searcher;
|
|
||||||
this.executor = executor;
|
|
||||||
}
|
|
||||||
|
|
||||||
SearchRewriteContext wrapSearcher() {
|
|
||||||
assert isWrapped == false : "searcher already wrapped";
|
|
||||||
isWrapped = true;
|
|
||||||
searcher = shard.wrapSearcher(searcher);
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
void execute(Runnable runnable) {
|
|
||||||
assert isWrapped : "searcher is not wrapped";
|
|
||||||
executor.execute(runnable);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public static final class CanMatchResponse extends SearchPhaseResult {
|
public static final class CanMatchResponse extends SearchPhaseResult {
|
||||||
private final boolean canMatch;
|
private final boolean canMatch;
|
||||||
private final MinAndMax<?> minAndMax;
|
private final MinAndMax<?> minAndMax;
|
||||||
|
|
|
@ -195,6 +195,26 @@ public class ShardSearchRequest extends TransportRequest implements IndicesReque
|
||||||
originalIndices = OriginalIndices.readOriginalIndices(in);
|
originalIndices = OriginalIndices.readOriginalIndices(in);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public ShardSearchRequest(ShardSearchRequest clone) {
|
||||||
|
this.shardId = clone.shardId;
|
||||||
|
this.searchType = clone.searchType;
|
||||||
|
this.numberOfShards = clone.numberOfShards;
|
||||||
|
this.scroll = clone.scroll;
|
||||||
|
this.source = clone.source;
|
||||||
|
this.types = clone.types;
|
||||||
|
this.aliasFilter = clone.aliasFilter;
|
||||||
|
this.indexBoost = clone.indexBoost;
|
||||||
|
this.nowInMillis = clone.nowInMillis;
|
||||||
|
this.requestCache = clone.requestCache;
|
||||||
|
this.clusterAlias = clone.clusterAlias;
|
||||||
|
this.allowPartialSearchResults = clone.allowPartialSearchResults;
|
||||||
|
this.indexRoutings = clone.indexRoutings;
|
||||||
|
this.preference = clone.preference;
|
||||||
|
this.canReturnNullResponseIfMatchNoDocs = clone.canReturnNullResponseIfMatchNoDocs;
|
||||||
|
this.bottomSortValues = clone.bottomSortValues;
|
||||||
|
this.originalIndices = clone.originalIndices;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void writeTo(StreamOutput out) throws IOException {
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
super.writeTo(out);
|
super.writeTo(out);
|
||||||
|
|
|
@ -364,11 +364,15 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
||||||
final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index"));
|
final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index"));
|
||||||
final IndexShard indexShard = indexService.getShard(0);
|
final IndexShard indexShard = indexService.getShard(0);
|
||||||
SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true);
|
SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true);
|
||||||
SearchService.SearchRewriteContext rewriteContext = service.acquireSearcherAndRewrite(
|
final SearchContext contextWithDefaultTimeout = service.createContext(
|
||||||
new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 1,
|
new ShardSearchRequest(
|
||||||
new AliasFilter(null, Strings.EMPTY_ARRAY), 1.0f, -1, null, null),
|
OriginalIndices.NONE,
|
||||||
indexShard);
|
searchRequest,
|
||||||
final SearchContext contextWithDefaultTimeout = service.createContext(rewriteContext, null);
|
indexShard.shardId(),
|
||||||
|
1,
|
||||||
|
new AliasFilter(null, Strings.EMPTY_ARRAY),
|
||||||
|
1.0f, -1, null, null
|
||||||
|
), null);
|
||||||
try {
|
try {
|
||||||
// the search context should inherit the default timeout
|
// the search context should inherit the default timeout
|
||||||
assertThat(contextWithDefaultTimeout.timeout(), equalTo(TimeValue.timeValueSeconds(5)));
|
assertThat(contextWithDefaultTimeout.timeout(), equalTo(TimeValue.timeValueSeconds(5)));
|
||||||
|
@ -379,11 +383,15 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
||||||
|
|
||||||
final long seconds = randomIntBetween(6, 10);
|
final long seconds = randomIntBetween(6, 10);
|
||||||
searchRequest.source(new SearchSourceBuilder().timeout(TimeValue.timeValueSeconds(seconds)));
|
searchRequest.source(new SearchSourceBuilder().timeout(TimeValue.timeValueSeconds(seconds)));
|
||||||
rewriteContext = service.acquireSearcherAndRewrite(
|
final SearchContext context = service.createContext(
|
||||||
new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 1,
|
new ShardSearchRequest(
|
||||||
new AliasFilter(null, Strings.EMPTY_ARRAY), 1.0f, -1, null, null),
|
OriginalIndices.NONE,
|
||||||
indexShard);
|
searchRequest,
|
||||||
final SearchContext context = service.createContext(rewriteContext, null);
|
indexShard.shardId(),
|
||||||
|
1,
|
||||||
|
new AliasFilter(null, Strings.EMPTY_ARRAY),
|
||||||
|
1.0f, -1, null, null
|
||||||
|
), null);
|
||||||
try {
|
try {
|
||||||
// the search context should inherit the query timeout
|
// the search context should inherit the query timeout
|
||||||
assertThat(context.timeout(), equalTo(TimeValue.timeValueSeconds(seconds)));
|
assertThat(context.timeout(), equalTo(TimeValue.timeValueSeconds(seconds)));
|
||||||
|
@ -411,22 +419,16 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
||||||
for (int i = 0; i < indexService.getIndexSettings().getMaxDocvalueFields(); i++) {
|
for (int i = 0; i < indexService.getIndexSettings().getMaxDocvalueFields(); i++) {
|
||||||
searchSourceBuilder.docValueField("field" + i);
|
searchSourceBuilder.docValueField("field" + i);
|
||||||
}
|
}
|
||||||
|
try (SearchContext context = service.createContext(
|
||||||
ShardSearchRequest shardRequest = new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 1,
|
new ShardSearchRequest(OriginalIndices.NONE,
|
||||||
new AliasFilter(null, Strings.EMPTY_ARRAY), 1.0f, -1, null, null);
|
searchRequest, indexShard.shardId(), 1,
|
||||||
|
new AliasFilter(null, Strings.EMPTY_ARRAY), 1.0f, -1, null, null
|
||||||
{
|
), null)) {
|
||||||
SearchService.SearchRewriteContext rewriteContext = service.acquireSearcherAndRewrite(shardRequest, indexShard);
|
|
||||||
try (SearchContext context = service.createContext(rewriteContext, null)) {
|
|
||||||
assertNotNull(context);
|
assertNotNull(context);
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
{
|
|
||||||
SearchService.SearchRewriteContext rewriteContext = service.acquireSearcherAndRewrite(shardRequest, indexShard);
|
|
||||||
searchSourceBuilder.docValueField("one_field_too_much");
|
searchSourceBuilder.docValueField("one_field_too_much");
|
||||||
IllegalArgumentException ex = expectThrows(IllegalArgumentException.class,
|
IllegalArgumentException ex = expectThrows(IllegalArgumentException.class,
|
||||||
() -> service.createContext(rewriteContext, null));
|
() -> service.createContext(new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 1,
|
||||||
|
new AliasFilter(null, Strings.EMPTY_ARRAY), 1.0f, -1, null, null), null));
|
||||||
assertEquals(
|
assertEquals(
|
||||||
"Trying to retrieve too many docvalue_fields. Must be less than or equal to: [100] but was [101]. "
|
"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());
|
||||||
|
@ -452,23 +454,15 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
||||||
searchSourceBuilder.scriptField("field" + i,
|
searchSourceBuilder.scriptField("field" + i,
|
||||||
new Script(ScriptType.INLINE, MockScriptEngine.NAME, CustomScriptPlugin.DUMMY_SCRIPT, Collections.emptyMap()));
|
new Script(ScriptType.INLINE, MockScriptEngine.NAME, CustomScriptPlugin.DUMMY_SCRIPT, Collections.emptyMap()));
|
||||||
}
|
}
|
||||||
|
try (SearchContext context = service.createContext(new ShardSearchRequest(OriginalIndices.NONE, searchRequest,
|
||||||
ShardSearchRequest shardRequest = new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 1,
|
indexShard.shardId(), 1, new AliasFilter(null, Strings.EMPTY_ARRAY),
|
||||||
new AliasFilter(null, Strings.EMPTY_ARRAY), 1.0f, -1, null, null);
|
1.0f, -1, null, null), null)) {
|
||||||
|
|
||||||
{
|
|
||||||
SearchService.SearchRewriteContext rewriteContext = service.acquireSearcherAndRewrite(shardRequest, indexShard);
|
|
||||||
try (SearchContext context = service.createContext(rewriteContext, null)) {
|
|
||||||
assertNotNull(context);
|
assertNotNull(context);
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
{
|
|
||||||
searchSourceBuilder.scriptField("anotherScriptField",
|
searchSourceBuilder.scriptField("anotherScriptField",
|
||||||
new Script(ScriptType.INLINE, MockScriptEngine.NAME, CustomScriptPlugin.DUMMY_SCRIPT, Collections.emptyMap()));
|
new Script(ScriptType.INLINE, MockScriptEngine.NAME, CustomScriptPlugin.DUMMY_SCRIPT, Collections.emptyMap()));
|
||||||
SearchService.SearchRewriteContext rewriteContext = service.acquireSearcherAndRewrite(shardRequest, indexShard);
|
|
||||||
IllegalArgumentException ex = expectThrows(IllegalArgumentException.class,
|
IllegalArgumentException ex = expectThrows(IllegalArgumentException.class,
|
||||||
() -> service.createContext(rewriteContext, null));
|
() -> service.createContext(new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 1,
|
||||||
|
new AliasFilter(null, Strings.EMPTY_ARRAY), 1.0f, -1, null, null), null));
|
||||||
assertEquals(
|
assertEquals(
|
||||||
"Trying to retrieve too many script_fields. Must be less than or equal to: [" + maxScriptFields + "] but was ["
|
"Trying to retrieve too many script_fields. Must be less than or equal to: [" + maxScriptFields + "] but was ["
|
||||||
+ (maxScriptFields + 1)
|
+ (maxScriptFields + 1)
|
||||||
|
@ -490,11 +484,9 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
||||||
searchSourceBuilder.scriptField("field" + 0,
|
searchSourceBuilder.scriptField("field" + 0,
|
||||||
new Script(ScriptType.INLINE, MockScriptEngine.NAME, CustomScriptPlugin.DUMMY_SCRIPT, Collections.emptyMap()));
|
new Script(ScriptType.INLINE, MockScriptEngine.NAME, CustomScriptPlugin.DUMMY_SCRIPT, Collections.emptyMap()));
|
||||||
searchSourceBuilder.size(0);
|
searchSourceBuilder.size(0);
|
||||||
SearchService.SearchRewriteContext rewriteContext = service.acquireSearcherAndRewrite(
|
try (SearchContext context = service.createContext(new ShardSearchRequest(OriginalIndices.NONE,
|
||||||
new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 1,
|
searchRequest, indexShard.shardId(), 1, new AliasFilter(null, Strings.EMPTY_ARRAY),
|
||||||
new AliasFilter(null, Strings.EMPTY_ARRAY), 1.0f, -1, null, null),
|
1.0f, -1, null, null), null)) {
|
||||||
indexShard);
|
|
||||||
try (SearchContext context = service.createContext(rewriteContext, null)) {
|
|
||||||
assertEquals(0, context.scriptFields().fields().size());
|
assertEquals(0, context.scriptFields().fields().size());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -502,7 +494,7 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
||||||
/**
|
/**
|
||||||
* test that creating more than the allowed number of scroll contexts throws an exception
|
* test that creating more than the allowed number of scroll contexts throws an exception
|
||||||
*/
|
*/
|
||||||
public void testMaxOpenScrollContexts() throws RuntimeException, IOException {
|
public void testMaxOpenScrollContexts() throws RuntimeException {
|
||||||
createIndex("index");
|
createIndex("index");
|
||||||
client().prepareIndex("index", "type", "1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get();
|
client().prepareIndex("index", "type", "1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get();
|
||||||
|
|
||||||
|
@ -528,10 +520,8 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
||||||
client().prepareSearch("index").setSize(1).setScroll("1m").get();
|
client().prepareSearch("index").setSize(1).setScroll("1m").get();
|
||||||
}
|
}
|
||||||
|
|
||||||
SearchService.SearchRewriteContext rewriteContext =
|
|
||||||
service.acquireSearcherAndRewrite(new ShardScrollRequestTest(indexShard.shardId()), indexShard);
|
|
||||||
ElasticsearchException ex = expectThrows(ElasticsearchException.class,
|
ElasticsearchException ex = expectThrows(ElasticsearchException.class,
|
||||||
() -> service.createAndPutContext(rewriteContext, null));
|
() -> service.createAndPutContext(new ShardScrollRequestTest(indexShard.shardId()), null));
|
||||||
assertEquals(
|
assertEquals(
|
||||||
"Trying to create too many scroll contexts. Must be less than or equal to: [" +
|
"Trying to create too many scroll contexts. Must be less than or equal to: [" +
|
||||||
SearchService.MAX_OPEN_SCROLL_CONTEXT.get(Settings.EMPTY) + "]. " +
|
SearchService.MAX_OPEN_SCROLL_CONTEXT.get(Settings.EMPTY) + "]. " +
|
||||||
|
@ -554,10 +544,8 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
||||||
try {
|
try {
|
||||||
latch.await();
|
latch.await();
|
||||||
for (; ; ) {
|
for (; ; ) {
|
||||||
SearchService.SearchRewriteContext rewriteContext =
|
|
||||||
searchService.acquireSearcherAndRewrite(new ShardScrollRequestTest(indexShard.shardId()), indexShard);
|
|
||||||
try {
|
try {
|
||||||
searchService.createAndPutContext(rewriteContext, null);
|
searchService.createAndPutContext(new ShardScrollRequestTest(indexShard.shardId()), null);
|
||||||
} catch (ElasticsearchException e) {
|
} catch (ElasticsearchException e) {
|
||||||
assertThat(e.getMessage(), equalTo(
|
assertThat(e.getMessage(), equalTo(
|
||||||
"Trying to create too many scroll contexts. Must be less than or equal to: " +
|
"Trying to create too many scroll contexts. Must be less than or equal to: " +
|
||||||
|
@ -657,7 +645,6 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
||||||
final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index"));
|
final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index"));
|
||||||
final IndexShard indexShard = indexService.getShard(0);
|
final IndexShard indexShard = indexService.getShard(0);
|
||||||
SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true);
|
SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true);
|
||||||
int numWrapReader = numWrapInvocations.get();
|
|
||||||
assertTrue(service.canMatch(new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 1,
|
assertTrue(service.canMatch(new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 1,
|
||||||
new AliasFilter(null, Strings.EMPTY_ARRAY), 1f, -1, null, null)).canMatch());
|
new AliasFilter(null, Strings.EMPTY_ARRAY), 1f, -1, null, null)).canMatch());
|
||||||
|
|
||||||
|
@ -681,7 +668,7 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
||||||
searchRequest.source(new SearchSourceBuilder().query(new MatchNoneQueryBuilder()));
|
searchRequest.source(new SearchSourceBuilder().query(new MatchNoneQueryBuilder()));
|
||||||
assertFalse(service.canMatch(new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 1,
|
assertFalse(service.canMatch(new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 1,
|
||||||
new AliasFilter(null, Strings.EMPTY_ARRAY), 1f, -1, null, null)).canMatch());
|
new AliasFilter(null, Strings.EMPTY_ARRAY), 1f, -1, null, null)).canMatch());
|
||||||
assertEquals(numWrapReader, numWrapInvocations.get());
|
assertEquals(0, numWrapInvocations.get());
|
||||||
|
|
||||||
ShardSearchRequest request = new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 1,
|
ShardSearchRequest request = new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 1,
|
||||||
new AliasFilter(null, Strings.EMPTY_ARRAY), 1.0f, -1, null, null);
|
new AliasFilter(null, Strings.EMPTY_ARRAY), 1.0f, -1, null, null);
|
||||||
|
@ -693,7 +680,7 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
||||||
public void onResponse(SearchPhaseResult searchPhaseResult) {
|
public void onResponse(SearchPhaseResult searchPhaseResult) {
|
||||||
try {
|
try {
|
||||||
// make sure that the wrapper is called when the query is actually executed
|
// make sure that the wrapper is called when the query is actually executed
|
||||||
assertEquals(numWrapReader+1, numWrapInvocations.get());
|
assertEquals(1, numWrapInvocations.get());
|
||||||
} finally {
|
} finally {
|
||||||
latch.countDown();
|
latch.countDown();
|
||||||
}
|
}
|
||||||
|
@ -826,16 +813,15 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
||||||
final ShardId shardId = new ShardId(indexService.index(), 0);
|
final ShardId shardId = new ShardId(indexService.index(), 0);
|
||||||
IndexShard indexShard = indexService.getShard(0);
|
IndexShard indexShard = indexService.getShard(0);
|
||||||
|
|
||||||
SearchService.SearchRewriteContext rewriteContext = service.acquireSearcherAndRewrite(
|
NullPointerException e = expectThrows(NullPointerException.class,
|
||||||
|
() -> service.createContext(
|
||||||
new ShardSearchRequest(shardId, null, 0, AliasFilter.EMPTY) {
|
new ShardSearchRequest(shardId, null, 0, AliasFilter.EMPTY) {
|
||||||
@Override
|
@Override
|
||||||
public SearchType searchType() {
|
public SearchType searchType() {
|
||||||
// induce an artificial NPE
|
// induce an artificial NPE
|
||||||
throw new NullPointerException("expected");
|
throw new NullPointerException("expected");
|
||||||
}
|
}
|
||||||
}, indexShard);
|
}, null));
|
||||||
NullPointerException e = expectThrows(NullPointerException.class,
|
|
||||||
() -> service.createContext(rewriteContext, null));
|
|
||||||
assertEquals("expected", e.getMessage());
|
assertEquals("expected", e.getMessage());
|
||||||
assertEquals("should have 2 store refs (IndexService + InternalEngine)", 2, indexService.getShard(0).store().refCount());
|
assertEquals("should have 2 store refs (IndexService + InternalEngine)", 2, indexService.getShard(0).store().refCount());
|
||||||
}
|
}
|
||||||
|
@ -987,8 +973,7 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
||||||
List<SearchContextId> contextIds = new ArrayList<>();
|
List<SearchContextId> contextIds = new ArrayList<>();
|
||||||
int numContexts = randomIntBetween(1, 10);
|
int numContexts = randomIntBetween(1, 10);
|
||||||
for (int i = 0; i < numContexts; i++) {
|
for (int i = 0; i < numContexts; i++) {
|
||||||
SearchService.SearchRewriteContext rewriteContext = searchService.acquireSearcherAndRewrite(shardSearchRequest, indexShard);
|
final SearchContext searchContext = searchService.createContext(shardSearchRequest, null);
|
||||||
final SearchContext searchContext = searchService.createContext(rewriteContext, null);
|
|
||||||
assertThat(searchContext.id().getId(), equalTo((long) (i + 1)));
|
assertThat(searchContext.id().getId(), equalTo((long) (i + 1)));
|
||||||
searchService.putContext(searchContext);
|
searchService.putContext(searchContext);
|
||||||
contextIds.add(searchContext.id());
|
contextIds.add(searchContext.id());
|
||||||
|
|
|
@ -60,26 +60,15 @@ public class ShardSearchRequestTests extends AbstractSearchTestCase {
|
||||||
ShardSearchRequest shardSearchTransportRequest = createShardSearchRequest();
|
ShardSearchRequest shardSearchTransportRequest = createShardSearchRequest();
|
||||||
ShardSearchRequest deserializedRequest =
|
ShardSearchRequest deserializedRequest =
|
||||||
copyWriteable(shardSearchTransportRequest, namedWriteableRegistry, ShardSearchRequest::new);
|
copyWriteable(shardSearchTransportRequest, namedWriteableRegistry, ShardSearchRequest::new);
|
||||||
assertEquals(deserializedRequest.scroll(), shardSearchTransportRequest.scroll());
|
assertEquals(shardSearchTransportRequest, deserializedRequest);
|
||||||
assertEquals(deserializedRequest.getAliasFilter(), shardSearchTransportRequest.getAliasFilter());
|
}
|
||||||
assertArrayEquals(deserializedRequest.indices(), shardSearchTransportRequest.indices());
|
|
||||||
assertArrayEquals(deserializedRequest.types(), shardSearchTransportRequest.types());
|
public void testClone() throws Exception {
|
||||||
assertEquals(deserializedRequest.indicesOptions(), shardSearchTransportRequest.indicesOptions());
|
for (int i = 0; i < 10; i++) {
|
||||||
assertEquals(deserializedRequest.nowInMillis(), shardSearchTransportRequest.nowInMillis());
|
ShardSearchRequest shardSearchTransportRequest = createShardSearchRequest();
|
||||||
assertEquals(deserializedRequest.source(), shardSearchTransportRequest.source());
|
ShardSearchRequest clone = new ShardSearchRequest(shardSearchTransportRequest);
|
||||||
assertEquals(deserializedRequest.searchType(), shardSearchTransportRequest.searchType());
|
assertEquals(shardSearchTransportRequest, clone);
|
||||||
assertEquals(deserializedRequest.shardId(), shardSearchTransportRequest.shardId());
|
}
|
||||||
assertEquals(deserializedRequest.numberOfShards(), shardSearchTransportRequest.numberOfShards());
|
|
||||||
assertArrayEquals(deserializedRequest.indexRoutings(), shardSearchTransportRequest.indexRoutings());
|
|
||||||
assertEquals(deserializedRequest.preference(), shardSearchTransportRequest.preference());
|
|
||||||
assertEquals(deserializedRequest.cacheKey(), shardSearchTransportRequest.cacheKey());
|
|
||||||
assertNotSame(deserializedRequest, shardSearchTransportRequest);
|
|
||||||
assertEquals(deserializedRequest.getAliasFilter(), shardSearchTransportRequest.getAliasFilter());
|
|
||||||
assertEquals(deserializedRequest.indexBoost(), shardSearchTransportRequest.indexBoost(), 0.0f);
|
|
||||||
assertEquals(deserializedRequest.getClusterAlias(), shardSearchTransportRequest.getClusterAlias());
|
|
||||||
assertEquals(shardSearchTransportRequest.allowPartialSearchResults(), deserializedRequest.allowPartialSearchResults());
|
|
||||||
assertEquals(deserializedRequest.canReturnNullResponseIfMatchNoDocs(),
|
|
||||||
shardSearchTransportRequest.canReturnNullResponseIfMatchNoDocs());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testAllowPartialResultsSerializationPre7_0_0() throws IOException {
|
public void testAllowPartialResultsSerializationPre7_0_0() throws IOException {
|
||||||
|
@ -161,6 +150,28 @@ public class ShardSearchRequestTests extends AbstractSearchTestCase {
|
||||||
expectThrows(InvalidAliasNameException.class, () -> aliasFilter(finalIndexMetadata, "unknown"));
|
expectThrows(InvalidAliasNameException.class, () -> aliasFilter(finalIndexMetadata, "unknown"));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static void assertEquals(ShardSearchRequest orig, ShardSearchRequest copy) throws IOException {
|
||||||
|
assertEquals(orig.scroll(), copy.scroll());
|
||||||
|
assertEquals(orig.getAliasFilter(), copy.getAliasFilter());
|
||||||
|
assertArrayEquals(orig.indices(), copy.indices());
|
||||||
|
assertEquals(orig.indicesOptions(), copy.indicesOptions());
|
||||||
|
assertEquals(orig.nowInMillis(), copy.nowInMillis());
|
||||||
|
assertEquals(orig.source(), copy.source());
|
||||||
|
assertEquals(orig.searchType(), copy.searchType());
|
||||||
|
assertEquals(orig.shardId(), copy.shardId());
|
||||||
|
assertEquals(orig.numberOfShards(), copy.numberOfShards());
|
||||||
|
assertArrayEquals(orig.indexRoutings(), copy.indexRoutings());
|
||||||
|
assertEquals(orig.preference(), copy.preference());
|
||||||
|
assertEquals(orig.cacheKey(), copy.cacheKey());
|
||||||
|
assertNotSame(orig, copy);
|
||||||
|
assertEquals(orig.getAliasFilter(), copy.getAliasFilter());
|
||||||
|
assertEquals(orig.indexBoost(), copy.indexBoost(), 0.0f);
|
||||||
|
assertEquals(orig.getClusterAlias(), copy.getClusterAlias());
|
||||||
|
assertEquals(orig.allowPartialSearchResults(), copy.allowPartialSearchResults());
|
||||||
|
assertEquals(orig.canReturnNullResponseIfMatchNoDocs(),
|
||||||
|
orig.canReturnNullResponseIfMatchNoDocs());
|
||||||
|
}
|
||||||
|
|
||||||
public static CompressedXContent filter(QueryBuilder filterBuilder) throws IOException {
|
public static CompressedXContent filter(QueryBuilder filterBuilder) throws IOException {
|
||||||
XContentBuilder builder = XContentFactory.jsonBuilder();
|
XContentBuilder builder = XContentFactory.jsonBuilder();
|
||||||
filterBuilder.toXContent(builder, ToXContent.EMPTY_PARAMS);
|
filterBuilder.toXContent(builder, ToXContent.EMPTY_PARAMS);
|
||||||
|
|
Loading…
Reference in New Issue