From 25ace7cb50abee7f7d8aaf57219bca42cadd3fdf Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 11 Jul 2017 20:36:13 +0200 Subject: [PATCH 01/12] Add a shard filter search phase to pre-filter shards based on query rewriting Today if we search across a large amount of shards we hit every shard. Yet, it's quite common to search across an index pattern for time based indices but filtering will exclude all results outside a certain time range ie. `now-3d`. While the search can potentially hit hunderets of shards the majority of the shards might yield 0 results since there is not document that is within this date range. Kibana for instance does this regularly but used `_field_stats` to optimzie the indice they need to query. Now with the deprecation of `_field_stats` and it's upcoming removal a single dashboard in kibanan can potentially turn into searches hitting hunderets or thousands of shards and that can easily cause search rejections even though the most of the requests are very likely super cheap and only need a query rewriting to early terminate with 0 results. This change adds a pre-filter phase for searches that can, if the number of shards are higher than a the `pre_filter_shards_after` threshold (defaults to 128 shards), fan out to the shards and check if the query can potentially match any documents at all. While false positives are possible, a negative response means that no matches are possible. These requests are not subject to rejection and can greatly reduce the number of shards a request needs to hit. The approach here is preferable to the kibana approach with field stats since it correctly handles aliases and uses the correct threadpools to execute these requests. Further it's completely transparent to the user and improves scalability of elasticsearch in general on large clusters. --- .../search/AbstractSearchAsyncAction.java | 1 - .../search/CanMatchPreFilterSearchPhase.java | 138 ++++++++++++++++++ .../action/search/DfsQueryPhase.java | 6 +- .../action/search/FetchSearchPhase.java | 5 +- .../action/search/InitialSearchPhase.java | 57 ++++++-- .../SearchDfsQueryThenFetchAsyncAction.java | 4 +- .../action/search/SearchPhaseController.java | 10 +- .../action/search/SearchRequest.java | 32 +++- .../action/search/SearchRequestBuilder.java | 11 ++ .../action/search/SearchTransportService.java | 50 ++++++- .../search/TransportMultiSearchAction.java | 1 + .../action/search/TransportSearchAction.java | 55 ++++--- .../action/search/RestMultiSearchAction.java | 10 +- .../rest/action/search/RestSearchAction.java | 1 + .../elasticsearch/search/SearchService.java | 12 ++ .../AbstractSearchAsyncActionTests.java | 2 +- .../action/search/FetchSearchPhaseTests.java | 12 +- .../action/search/SearchAsyncActionTests.java | 6 +- .../search/SearchPhaseControllerTests.java | 12 +- .../resources/rest-api-spec/api/msearch.json | 5 + .../resources/rest-api-spec/api/search.json | 5 + .../search/140_pre_filter_search_shards.yml | 101 +++++++++++++ .../test/client/RandomizingClient.java | 9 ++ 23 files changed, 479 insertions(+), 66 deletions(-) create mode 100644 core/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java create mode 100644 rest-api-spec/src/main/resources/rest-api-spec/test/search/140_pre_filter_search_shards.yml diff --git a/core/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/core/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index e074c2bad039b..173da599671a0 100644 --- a/core/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -31,7 +31,6 @@ import org.elasticsearch.cluster.routing.GroupShardsIterator; import org.elasticsearch.common.Nullable; 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.AliasFilter; diff --git a/core/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java b/core/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java new file mode 100644 index 0000000000000..9caf2c4d2cf86 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java @@ -0,0 +1,138 @@ +/* + * 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.apache.logging.log4j.Logger; +import org.apache.lucene.util.FixedBitSet; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.cluster.routing.GroupShardsIterator; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.search.internal.AliasFilter; +import org.elasticsearch.transport.Transport; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Executor; +import java.util.function.BiFunction; +import java.util.function.Function; +import java.util.stream.Stream; + +/** + * This search phrase can be used as an initial search phase to pre-filter search shards based on query rewriting. + * The queries are rewritten against the shards and based on the rewrite result shards might be able to be excluded + * from the search. The extra round trip to the search shards is very cheap and is not subject to rejections + * which allows to fan out to more shards at the same time without running into rejections even if we are hitting a + * large portion of the clusters indices. + */ +final class CanMatchPreFilterSearchPhase extends AbstractSearchAsyncAction { + + private final Function, SearchPhase> phaseFactory; + private final GroupShardsIterator shardsIts; + + CanMatchPreFilterSearchPhase(Logger logger, SearchTransportService searchTransportService, + BiFunction nodeIdToConnection, + Map aliasFilter, Map concreteIndexBoosts, + Executor executor, SearchRequest request, + ActionListener listener, GroupShardsIterator shardsIts, + TransportSearchAction.SearchTimeProvider timeProvider, long clusterStateVersion, + SearchTask task, Function, SearchPhase> phaseFactory) { + super("can_match", logger, searchTransportService, nodeIdToConnection, aliasFilter, concreteIndexBoosts, executor, request, + listener, + shardsIts, timeProvider, clusterStateVersion, task, new BitSetSearchPhaseResults(shardsIts.size())); + this.phaseFactory = phaseFactory; + this.shardsIts = shardsIts; + } + + @Override + protected void executePhaseOnShard(SearchShardIterator shardIt, ShardRouting shard, + SearchActionListener listener) { + getSearchTransport().sendCanMatch(getConnection(shardIt.getClusterAlias(), shard.currentNodeId()), + buildShardSearchRequest(shardIt), getTask(), listener); + } + + @Override + protected SearchPhase getNextPhase(SearchPhaseResults results, + SearchPhaseContext context) { + return phaseFactory.apply(getIterator((BitSetSearchPhaseResults) results, shardsIts)); + } + + private GroupShardsIterator getIterator(BitSetSearchPhaseResults results, + GroupShardsIterator shardsIts) { + if (results.numMatches == shardsIts.size()) { + shardsIts.iterator().forEachRemaining(i -> i.reset()); + return shardsIts; + } else if (results.numMatches == 0) { + // this is a special case where we have no hit but we need to get at least one search response in order + // to produce a valid search result with all the aggs etc. at least that is what I think is the case... and clint does so + // too :D + Iterator iterator = shardsIts.iterator(); + if (iterator.hasNext()) { + SearchShardIterator next = iterator.next(); + next.reset(); + return new GroupShardsIterator<>(Collections.singletonList(next)); + } + } + List shardIterators = new ArrayList<>(results.numMatches); + int i = 0; + for (SearchShardIterator iter : shardsIts) { + if (results.possibleMatches.get(i++)) { + iter.reset(); + shardIterators.add(iter); + } + } + return new GroupShardsIterator<>(shardIterators); + } + + private static final class BitSetSearchPhaseResults extends InitialSearchPhase. + SearchPhaseResults { + + private FixedBitSet possibleMatches; + private int numMatches; + + BitSetSearchPhaseResults(int size) { + super(size); + possibleMatches = new FixedBitSet(size); + } + + @Override + void consumeResult(SearchTransportService.CanMatchResponse result) { + if (result.canMatch()) { + synchronized (possibleMatches) { + possibleMatches.set(result.getShardIndex()); + numMatches++; + } + } + } + + @Override + boolean hasResult(int shardIndex) { + synchronized (possibleMatches) { + return possibleMatches.get(shardIndex); + } + } + + @Override + Stream getSuccessfulResults() { + return Stream.empty(); + } + } +} diff --git a/core/src/main/java/org/elasticsearch/action/search/DfsQueryPhase.java b/core/src/main/java/org/elasticsearch/action/search/DfsQueryPhase.java index a72dcac4f241a..db0425db7c320 100644 --- a/core/src/main/java/org/elasticsearch/action/search/DfsQueryPhase.java +++ b/core/src/main/java/org/elasticsearch/action/search/DfsQueryPhase.java @@ -41,16 +41,16 @@ * @see CountedCollector#onFailure(int, SearchShardTarget, Exception) */ final class DfsQueryPhase extends SearchPhase { - private final InitialSearchPhase.SearchPhaseResults queryResult; + private final InitialSearchPhase.ArraySearchPhaseResults queryResult; private final SearchPhaseController searchPhaseController; private final AtomicArray dfsSearchResults; - private final Function, SearchPhase> nextPhaseFactory; + private final Function, SearchPhase> nextPhaseFactory; private final SearchPhaseContext context; private final SearchTransportService searchTransportService; DfsQueryPhase(AtomicArray dfsSearchResults, SearchPhaseController searchPhaseController, - Function, SearchPhase> nextPhaseFactory, + Function, SearchPhase> nextPhaseFactory, SearchPhaseContext context) { super("dfs_query"); this.queryResult = searchPhaseController.newSearchPhaseResults(context.getRequest(), context.getNumShards()); diff --git a/core/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java b/core/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java index c26fc63421d17..e5fdf2eff725e 100644 --- a/core/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java +++ b/core/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java @@ -69,7 +69,7 @@ final class FetchSearchPhase extends SearchPhase { } this.fetchResults = new AtomicArray<>(resultConsumer.getNumShards()); this.searchPhaseController = searchPhaseController; - this.queryResults = resultConsumer.results; + this.queryResults = resultConsumer.getAtomicArray(); this.nextPhaseFactory = nextPhaseFactory; this.context = context; this.logger = context.getLogger(); @@ -105,7 +105,8 @@ private void innerRun() throws IOException { -> moveToNextPhase(searchPhaseController, scrollId, reducedQueryPhase, queryAndFetchOptimization ? queryResults : fetchResults); if (queryAndFetchOptimization) { - assert phaseResults.isEmpty() || phaseResults.get(0).fetchResult() != null; + assert phaseResults.isEmpty() || phaseResults.get(0).fetchResult() != null : "phaseResults emtpy [" + phaseResults.isEmpty() + + "], single result: " + phaseResults.get(0).fetchResult(); // query AND fetch optimization finishPhase.run(); } else { diff --git a/core/src/main/java/org/elasticsearch/action/search/InitialSearchPhase.java b/core/src/main/java/org/elasticsearch/action/search/InitialSearchPhase.java index 1636236525015..65892969ac3b3 100644 --- a/core/src/main/java/org/elasticsearch/action/search/InitialSearchPhase.java +++ b/core/src/main/java/org/elasticsearch/action/search/InitialSearchPhase.java @@ -227,41 +227,37 @@ protected abstract void executePhaseOnShard(SearchShardIterator shardIt, ShardRo /** * This class acts as a basic result collection that can be extended to do on-the-fly reduction or result processing */ - static class SearchPhaseResults { - final AtomicArray results; + abstract static class SearchPhaseResults { + private final int numShards; - SearchPhaseResults(int size) { - results = new AtomicArray<>(size); + protected SearchPhaseResults(int numShards) { + this.numShards = numShards; } - /** * Returns the number of expected results this class should collect */ final int getNumShards() { - return results.length(); + return numShards; } /** * A stream of all non-null (successful) shard results */ - final Stream getSuccessfulResults() { - return results.asList().stream(); - } + abstract Stream getSuccessfulResults(); /** * Consumes a single shard result * @param result the shards result */ - void consumeResult(Result result) { - assert results.get(result.getShardIndex()) == null : "shardIndex: " + result.getShardIndex() + " is already set"; - results.set(result.getShardIndex(), result); - } + abstract void consumeResult(Result result); /** * Returns true iff a result if present for the given shard ID. */ - final boolean hasResult(int shardIndex) { - return results.get(shardIndex) != null; + abstract boolean hasResult(int shardIndex); + + AtomicArray getAtomicArray() { + throw new UnsupportedOperationException(); } /** @@ -271,4 +267,35 @@ SearchPhaseController.ReducedQueryPhase reduce() { throw new UnsupportedOperationException("reduce is not supported"); } } + + /** + * This class acts as a basic result collection that can be extended to do on-the-fly reduction or result processing + */ + static class ArraySearchPhaseResults extends SearchPhaseResults { + final AtomicArray results; + + ArraySearchPhaseResults(int size) { + super(size); + this.results = new AtomicArray<>(size); + } + + Stream getSuccessfulResults() { + return results.asList().stream(); + } + + void consumeResult(Result result) { + assert results.get(result.getShardIndex()) == null : "shardIndex: " + result.getShardIndex() + " is already set"; + results.set(result.getShardIndex(), result); + } + + boolean hasResult(int shardIndex) { + return results.get(shardIndex) != null; + } + + @Override + AtomicArray getAtomicArray() { + return results; + } + } + } diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchDfsQueryThenFetchAsyncAction.java b/core/src/main/java/org/elasticsearch/action/search/SearchDfsQueryThenFetchAsyncAction.java index a87b58c4e67b1..a901d71157137 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchDfsQueryThenFetchAsyncAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchDfsQueryThenFetchAsyncAction.java @@ -42,7 +42,7 @@ final class SearchDfsQueryThenFetchAsyncAction extends AbstractSearchAsyncAction final GroupShardsIterator shardsIts, final TransportSearchAction.SearchTimeProvider timeProvider, final long clusterStateVersion, final SearchTask task) { super("dfs", logger, searchTransportService, nodeIdToConnection, aliasFilter, concreteIndexBoosts, executor, request, listener, - shardsIts, timeProvider, clusterStateVersion, task, new SearchPhaseResults<>(shardsIts.size())); + shardsIts, timeProvider, clusterStateVersion, task, new ArraySearchPhaseResults<>(shardsIts.size())); this.searchPhaseController = searchPhaseController; } @@ -55,7 +55,7 @@ protected void executePhaseOnShard(final SearchShardIterator shardIt, final Shar @Override protected SearchPhase getNextPhase(final SearchPhaseResults results, final SearchPhaseContext context) { - return new DfsQueryPhase(results.results, searchPhaseController, (queryResults) -> + return new DfsQueryPhase(results.getAtomicArray(), searchPhaseController, (queryResults) -> new FetchSearchPhase(queryResults, searchPhaseController, context), context); } } diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java b/core/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java index 879607d059e80..69a1f6eb151b8 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java @@ -606,12 +606,12 @@ public InternalSearchResponse buildResponse(SearchHits hits) { } /** - * A {@link org.elasticsearch.action.search.InitialSearchPhase.SearchPhaseResults} implementation + * A {@link InitialSearchPhase.ArraySearchPhaseResults} implementation * that incrementally reduces aggregation results as shard results are consumed. * This implementation can be configured to batch up a certain amount of results and only reduce them * iff the buffer is exhausted. */ - static final class QueryPhaseResultConsumer extends InitialSearchPhase.SearchPhaseResults { + static final class QueryPhaseResultConsumer extends InitialSearchPhase.ArraySearchPhaseResults { private final InternalAggregations[] aggsBuffer; private final TopDocs[] topDocsBuffer; private final boolean hasAggs; @@ -713,9 +713,9 @@ int getNumBuffered() { } /** - * Returns a new SearchPhaseResults instance. This might return an instance that reduces search responses incrementally. + * Returns a new ArraySearchPhaseResults instance. This might return an instance that reduces search responses incrementally. */ - InitialSearchPhase.SearchPhaseResults newSearchPhaseResults(SearchRequest request, int numShards) { + InitialSearchPhase.ArraySearchPhaseResults newSearchPhaseResults(SearchRequest request, int numShards) { SearchSourceBuilder source = request.source(); boolean isScrollRequest = request.scroll() != null; final boolean hasAggs = source != null && source.aggregations() != null; @@ -729,7 +729,7 @@ InitialSearchPhase.SearchPhaseResults newSearchPhaseResults(S return new QueryPhaseResultConsumer(this, numShards, request.getBatchedReduceSize(), hasTopDocs, hasAggs); } } - return new InitialSearchPhase.SearchPhaseResults(numShards) { + return new InitialSearchPhase.ArraySearchPhaseResults(numShards) { @Override public ReducedQueryPhase reduce() { return reducedQueryPhase(results.asList(), isScrollRequest, trackTotalHits); diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchRequest.java b/core/src/main/java/org/elasticsearch/action/search/SearchRequest.java index 02cde220b3562..3522d2d7b9776 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchRequest.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchRequest.java @@ -58,6 +58,8 @@ public final class SearchRequest extends ActionRequest implements IndicesRequest private static final ToXContent.Params FORMAT_PARAMS = new ToXContent.MapParams(Collections.singletonMap("pretty", "false")); + public static final int DEFAULT_PRE_FILTER_SHARDS_AFTER = 128; + private SearchType searchType = SearchType.DEFAULT; private String[] indices = Strings.EMPTY_ARRAY; @@ -77,6 +79,8 @@ public final class SearchRequest extends ActionRequest implements IndicesRequest private int maxConcurrentShardRequests = 0; + private int preFilterShardsAfter = DEFAULT_PRE_FILTER_SHARDS_AFTER; + private String[] types = Strings.EMPTY_ARRAY; public static final IndicesOptions DEFAULT_INDICES_OPTIONS = IndicesOptions.strictExpandOpenAndForbidClosed(); @@ -325,6 +329,28 @@ public void setMaxConcurrentShardRequests(int maxConcurrentShardRequests) { } this.maxConcurrentShardRequests = maxConcurrentShardRequests; } + /** + * Sets a threshold that enforces a pre-filter roundtrip to prefilter search shards based on query rewriting if the number of shards + * the search request expands to exceeds the threshold. This filter roundtrip can limit the number of shards significantly if for + * instance a shard can not match any documents based on it's rewrite method ie. if date filters are mandatory to match but the shard + * bounds and the query are disjoint. The default is 128 + */ + public void setPreFilterSearchShardsAfter(int preFilterShardsAfter) { + if (preFilterShardsAfter < 1) { + throw new IllegalArgumentException("preFilterShardsAfter must be >= 1"); + } + this.preFilterShardsAfter = preFilterShardsAfter; + } + + /** + * Returns a threshold that enforces a pre-filter roundtrip to prefilter search shards based on query rewriting if the number of shards + * the search request expands to exceeds the threshold. This filter roundtrip can limit the number of shards significantly if for + * instance a shard can not match any documents based on it's rewrite method ie. if date filters are mandatory to match but the shard + * bounds and the query are disjoint. The default is 128 + */ + public int getPreFilterShardsAfter() { + return preFilterShardsAfter; + } /** * Returns true iff the maxConcurrentShardRequest is set. @@ -382,6 +408,7 @@ public void readFrom(StreamInput in) throws IOException { batchedReduceSize = in.readVInt(); if (in.getVersion().onOrAfter(Version.V_6_0_0_beta1)) { maxConcurrentShardRequests = in.readVInt(); + preFilterShardsAfter = in.readVInt(); } } @@ -403,6 +430,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeVInt(batchedReduceSize); if (out.getVersion().onOrAfter(Version.V_6_0_0_beta1)) { out.writeVInt(maxConcurrentShardRequests); + out.writeVInt(preFilterShardsAfter); } } @@ -425,13 +453,14 @@ public boolean equals(Object o) { Arrays.equals(types, that.types) && Objects.equals(batchedReduceSize, that.batchedReduceSize) && Objects.equals(maxConcurrentShardRequests, that.maxConcurrentShardRequests) && + Objects.equals(preFilterShardsAfter, that.preFilterShardsAfter) && Objects.equals(indicesOptions, that.indicesOptions); } @Override public int hashCode() { return Objects.hash(searchType, Arrays.hashCode(indices), routing, preference, source, requestCache, - scroll, Arrays.hashCode(types), indicesOptions, maxConcurrentShardRequests); + scroll, Arrays.hashCode(types), indicesOptions, batchedReduceSize, maxConcurrentShardRequests, preFilterShardsAfter); } @Override @@ -447,6 +476,7 @@ public String toString() { ", scroll=" + scroll + ", maxConcurrentShardRequests=" + maxConcurrentShardRequests + ", batchedReduceSize=" + batchedReduceSize + + ", preFilterShardsAfter=" + preFilterShardsAfter + ", source=" + source + '}'; } } diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchRequestBuilder.java b/core/src/main/java/org/elasticsearch/action/search/SearchRequestBuilder.java index 49e25f67493cb..f236c417c690d 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchRequestBuilder.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchRequestBuilder.java @@ -535,4 +535,15 @@ public SearchRequestBuilder setMaxConcurrentShardRequests(int maxConcurrentShard this.request.setMaxConcurrentShardRequests(maxConcurrentShardRequests); return this; } + + /** + * Sets a threshold that enforces a pre-filter roundtrip to prefilter search shards based on query rewriting if the number of shards + * the search request expands to exceeds the threshold. This filter roundtrip can limit the number of shards significantly if for + * instance a shard can not match any documents based on it's rewrite method ie. if date filters are mandatory to match but the shard + * bounds and the query are disjoint. The default is 128 + */ + public SearchRequestBuilder setPreFilterShardsAfter(int preFilterShardsAfter) { + this.request.setPreFilterSearchShardsAfter(preFilterShardsAfter); + return this; + } } diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index e75d52db3ef31..b2110bcd81f30 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -74,6 +74,7 @@ public class SearchTransportService extends AbstractComponent { public static final String QUERY_FETCH_SCROLL_ACTION_NAME = "indices:data/read/search[phase/query+fetch/scroll]"; public static final String FETCH_ID_SCROLL_ACTION_NAME = "indices:data/read/search[phase/fetch/id/scroll]"; public static final String FETCH_ID_ACTION_NAME = "indices:data/read/search[phase/fetch/id]"; + public static final String QUERY_CAN_MATCH_NAME = "indices:data/read/search[can_match]"; private final TransportService transportService; @@ -102,6 +103,12 @@ public void sendFreeContext(Transport.Connection connection, long contextId, fin TransportRequestOptions.EMPTY, new ActionListenerResponseHandler<>(listener, SearchFreeContextResponse::new)); } + public void sendCanMatch(Transport.Connection connection, final ShardSearchTransportRequest request, SearchTask task, final + ActionListener listener) { + transportService.sendChildRequest(connection, QUERY_CAN_MATCH_NAME, request, task, + TransportRequestOptions.EMPTY, new ActionListenerResponseHandler<>(listener, CanMatchResponse::new)); + } + public void sendClearAllScrollContexts(Transport.Connection connection, final ActionListener listener) { transportService.sendRequest(connection, CLEAR_SCROLL_CONTEXTS_ACTION_NAME, TransportRequest.Empty.INSTANCE, TransportRequestOptions.EMPTY, new ActionListenerResponseHandler<>(listener, () -> TransportResponse.Empty.INSTANCE)); @@ -285,8 +292,7 @@ public void messageReceived(SearchFreeContextRequest request, TransportChannel c }); TransportActionProxy.registerProxyAction(transportService, FREE_CONTEXT_ACTION_NAME, SearchFreeContextResponse::new); transportService.registerRequestHandler(CLEAR_SCROLL_CONTEXTS_ACTION_NAME, () -> TransportRequest.Empty.INSTANCE, - ThreadPool.Names.SAME, - new TaskAwareTransportRequestHandler() { + ThreadPool.Names.SAME, new TaskAwareTransportRequestHandler() { @Override public void messageReceived(TransportRequest.Empty request, TransportChannel channel, Task task) throws Exception { searchService.freeAllScrollContexts(); @@ -366,8 +372,48 @@ public void messageReceived(ShardFetchSearchRequest request, TransportChannel ch } }); TransportActionProxy.registerProxyAction(transportService, FETCH_ID_ACTION_NAME, FetchSearchResult::new); + + // this is super cheap and should not hit thread-pool rejections + transportService.registerRequestHandler(QUERY_CAN_MATCH_NAME, ShardSearchTransportRequest::new, ThreadPool.Names.SEARCH, + false, true, new TaskAwareTransportRequestHandler() { + @Override + public void messageReceived(ShardSearchTransportRequest request, TransportChannel channel, Task task) throws Exception { + boolean canMatch = searchService.canMatch(request); + channel.sendResponse(new CanMatchResponse(canMatch)); + } + }); + TransportActionProxy.registerProxyAction(transportService, QUERY_CAN_MATCH_NAME, QuerySearchResult::new); } + public static final class CanMatchResponse extends SearchPhaseResult { + private boolean canMatch; + + public CanMatchResponse() { + } + + public CanMatchResponse(boolean canMatch) { + this.canMatch = canMatch; + } + + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + canMatch = in.readBoolean(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeBoolean(canMatch); + } + + public boolean canMatch() { + return canMatch; + } + } + + /** * Returns a connection to the given node on the provided cluster. If the cluster alias is null the node will be resolved * against the local cluster. diff --git a/core/src/main/java/org/elasticsearch/action/search/TransportMultiSearchAction.java b/core/src/main/java/org/elasticsearch/action/search/TransportMultiSearchAction.java index db5a21edb2bea..b65cd4d55516a 100644 --- a/core/src/main/java/org/elasticsearch/action/search/TransportMultiSearchAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/TransportMultiSearchAction.java @@ -34,6 +34,7 @@ import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.util.List; import java.util.Queue; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.atomic.AtomicInteger; diff --git a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index 2c3900e9951fc..89c643071341e 100644 --- a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -50,6 +50,7 @@ import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportService; +import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -313,8 +314,11 @@ private void executeSearch(SearchTask task, SearchTimeProvider timeProvider, Sea searchRequest.setMaxConcurrentShardRequests(Math.min(256, nodeCount * IndexMetaData.INDEX_NUMBER_OF_SHARDS_SETTING.getDefault(Settings.EMPTY))); } + boolean preFilterSearchShards = searchRequest.searchType() == QUERY_THEN_FETCH && // we can't do this for DFS it needs to fan out + // to all shards all the time + searchRequest.getPreFilterShardsAfter() < shardIterators.size(); searchAsyncAction(task, searchRequest, shardIterators, timeProvider, connectionLookup, clusterState.version(), - Collections.unmodifiableMap(aliasFilter), concreteIndexBoosts, listener).start(); + Collections.unmodifiableMap(aliasFilter), concreteIndexBoosts, listener, preFilterSearchShards).start(); } static GroupShardsIterator mergeShardsIterators(GroupShardsIterator localShardsIterator, @@ -341,25 +345,40 @@ private AbstractSearchAsyncAction searchAsyncAction(SearchTask task, SearchReque BiFunction connectionLookup, long clusterStateVersion, Map aliasFilter, Map concreteIndexBoosts, - ActionListener listener) { + ActionListener listener, boolean preFilter) { Executor executor = threadPool.executor(ThreadPool.Names.SEARCH); - AbstractSearchAsyncAction searchAsyncAction; - switch(searchRequest.searchType()) { - case DFS_QUERY_THEN_FETCH: - searchAsyncAction = new SearchDfsQueryThenFetchAsyncAction(logger, searchTransportService, connectionLookup, - aliasFilter, concreteIndexBoosts, searchPhaseController, executor, searchRequest, listener, shardIterators, - timeProvider, clusterStateVersion, task); - break; - case QUERY_AND_FETCH: - case QUERY_THEN_FETCH: - searchAsyncAction = new SearchQueryThenFetchAsyncAction(logger, searchTransportService, connectionLookup, - aliasFilter, concreteIndexBoosts, searchPhaseController, executor, searchRequest, listener, shardIterators, - timeProvider, clusterStateVersion, task); - break; - default: - throw new IllegalStateException("Unknown search type: [" + searchRequest.searchType() + "]"); + if (preFilter) { + return new CanMatchPreFilterSearchPhase(logger, searchTransportService, connectionLookup, + aliasFilter, concreteIndexBoosts, executor, searchRequest, listener, shardIterators, + timeProvider, clusterStateVersion, task, (iter) -> { + AbstractSearchAsyncAction action = searchAsyncAction(task, searchRequest, iter, timeProvider, connectionLookup, + clusterStateVersion, aliasFilter, concreteIndexBoosts, listener, false); + return new SearchPhase(action.getName()) { + @Override + public void run() throws IOException { + action.start(); + } + }; + }); + } else { + AbstractSearchAsyncAction searchAsyncAction; + switch (searchRequest.searchType()) { + case DFS_QUERY_THEN_FETCH: + searchAsyncAction = new SearchDfsQueryThenFetchAsyncAction(logger, searchTransportService, connectionLookup, + aliasFilter, concreteIndexBoosts, searchPhaseController, executor, searchRequest, listener, shardIterators, + timeProvider, clusterStateVersion, task); + break; + case QUERY_AND_FETCH: + case QUERY_THEN_FETCH: + searchAsyncAction = new SearchQueryThenFetchAsyncAction(logger, searchTransportService, connectionLookup, + aliasFilter, concreteIndexBoosts, searchPhaseController, executor, searchRequest, listener, shardIterators, + timeProvider, clusterStateVersion, task); + break; + default: + throw new IllegalStateException("Unknown search type: [" + searchRequest.searchType() + "]"); + } + return searchAsyncAction; } - return searchAsyncAction; } private static void failIfOverShardCountLimit(ClusterService clusterService, int shardCount) { diff --git a/core/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java b/core/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java index 0d9c389d0c246..a18cb5f266402 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java @@ -39,6 +39,7 @@ import java.io.IOException; import java.util.Collections; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.function.BiConsumer; @@ -88,6 +89,9 @@ public static MultiSearchRequest parseRequest(RestRequest restRequest, boolean a multiRequest.maxConcurrentSearchRequests(restRequest.paramAsInt("max_concurrent_searches", 0)); } + int preFilterShardsAfter = restRequest.paramAsInt("pre_filter_shards_after", SearchRequest.DEFAULT_PRE_FILTER_SHARDS_AFTER); + + parseMultiLineRequest(restRequest, multiRequest.indicesOptions(), allowExplicitIndex, (searchRequest, parser) -> { try { searchRequest.source(SearchSourceBuilder.fromXContent(parser)); @@ -96,7 +100,11 @@ public static MultiSearchRequest parseRequest(RestRequest restRequest, boolean a throw new ElasticsearchParseException("Exception when parsing search request", e); } }); - + List requests = multiRequest.requests(); + preFilterShardsAfter = Math.max(1, preFilterShardsAfter / (requests.size()+1)); + for (SearchRequest request : requests) { + request.setPreFilterSearchShardsAfter(preFilterShardsAfter); + } return multiRequest; } diff --git a/core/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java b/core/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java index b871446ba8853..0973db28f8f43 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java @@ -98,6 +98,7 @@ public static void parseSearchRequest(SearchRequest searchRequest, RestRequest r final int batchedReduceSize = request.paramAsInt("batched_reduce_size", searchRequest.getBatchedReduceSize()); searchRequest.setBatchedReduceSize(batchedReduceSize); + searchRequest.setPreFilterSearchShardsAfter(request.paramAsInt("pre_filter_shards_after", searchRequest.getPreFilterShardsAfter())); if (request.hasParam("max_concurrent_shard_requests")) { // only set if we have the parameter since we auto adjust the max concurrency on the coordinator diff --git a/core/src/main/java/org/elasticsearch/search/SearchService.java b/core/src/main/java/org/elasticsearch/search/SearchService.java index 3188ca50d1026..8f192bccd1243 100644 --- a/core/src/main/java/org/elasticsearch/search/SearchService.java +++ b/core/src/main/java/org/elasticsearch/search/SearchService.java @@ -42,6 +42,7 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.query.InnerHitContextBuilder; +import org.elasticsearch.index.query.MatchNoneQueryBuilder; import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexShard; @@ -832,4 +833,15 @@ public void run() { public AliasFilter buildAliasFilter(ClusterState state, String index, String... expressions) { return indicesService.buildAliasFilter(state, index, expressions); } + + /** + * This method does a very quick rewrite of the query and returns true if the query can potentially match any documents. + * This method can have false positives while if it returns false the query won't match any documents on the current + * shard. + */ + public boolean canMatch(ShardSearchRequest request) throws IOException { + try (DefaultSearchContext context = createSearchContext(request, defaultSearchTimeout, null)) { + return context.request().source().query() instanceof MatchNoneQueryBuilder == false; + } + } } diff --git a/core/src/test/java/org/elasticsearch/action/search/AbstractSearchAsyncActionTests.java b/core/src/test/java/org/elasticsearch/action/search/AbstractSearchAsyncActionTests.java index 839a061f852aa..8445fb08fabd1 100644 --- a/core/src/test/java/org/elasticsearch/action/search/AbstractSearchAsyncActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/AbstractSearchAsyncActionTests.java @@ -64,7 +64,7 @@ private AbstractSearchAsyncAction createAction( Collections.singletonMap("foo", new AliasFilter(new MatchAllQueryBuilder())), Collections.singletonMap("foo", 2.0f), null, new SearchRequest(), null, new GroupShardsIterator<>(Collections.singletonList( new SearchShardIterator(null, null, Collections.emptyList(), null))), timeProvider, 0, null, - new InitialSearchPhase.SearchPhaseResults<>(10)) { + new InitialSearchPhase.ArraySearchPhaseResults<>(10)) { @Override protected SearchPhase getNextPhase(final SearchPhaseResults results, final SearchPhaseContext context) { return null; diff --git a/core/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java b/core/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java index 3f940c54e62b3..b36ef4e0b5105 100644 --- a/core/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java @@ -46,7 +46,7 @@ public class FetchSearchPhaseTests extends ESTestCase { public void testShortcutQueryAndFetchOptimization() throws IOException { SearchPhaseController controller = new SearchPhaseController(Settings.EMPTY, BigArrays.NON_RECYCLING_INSTANCE, null); MockSearchPhaseContext mockSearchPhaseContext = new MockSearchPhaseContext(1); - InitialSearchPhase.SearchPhaseResults results = + InitialSearchPhase.ArraySearchPhaseResults results = controller.newSearchPhaseResults(mockSearchPhaseContext.getRequest(), 1); AtomicReference responseRef = new AtomicReference<>(); boolean hasHits = randomBoolean(); @@ -86,7 +86,7 @@ public void run() throws IOException { public void testFetchTwoDocument() throws IOException { MockSearchPhaseContext mockSearchPhaseContext = new MockSearchPhaseContext(2); SearchPhaseController controller = new SearchPhaseController(Settings.EMPTY, BigArrays.NON_RECYCLING_INSTANCE, null); - InitialSearchPhase.SearchPhaseResults results = + InitialSearchPhase.ArraySearchPhaseResults results = controller.newSearchPhaseResults(mockSearchPhaseContext.getRequest(), 2); AtomicReference responseRef = new AtomicReference<>(); int resultSetSize = randomIntBetween(2, 10); @@ -140,7 +140,7 @@ public void run() throws IOException { public void testFailFetchOneDoc() throws IOException { MockSearchPhaseContext mockSearchPhaseContext = new MockSearchPhaseContext(2); SearchPhaseController controller = new SearchPhaseController(Settings.EMPTY, BigArrays.NON_RECYCLING_INSTANCE, null); - InitialSearchPhase.SearchPhaseResults results = + InitialSearchPhase.ArraySearchPhaseResults results = controller.newSearchPhaseResults(mockSearchPhaseContext.getRequest(), 2); AtomicReference responseRef = new AtomicReference<>(); int resultSetSize = randomIntBetween(2, 10); @@ -199,7 +199,7 @@ public void testFetchDocsConcurrently() throws IOException, InterruptedException int numHits = randomIntBetween(2, 100); // also numshards --> 1 hit per shard SearchPhaseController controller = new SearchPhaseController(Settings.EMPTY, BigArrays.NON_RECYCLING_INSTANCE, null); MockSearchPhaseContext mockSearchPhaseContext = new MockSearchPhaseContext(numHits); - InitialSearchPhase.SearchPhaseResults results = + InitialSearchPhase.ArraySearchPhaseResults results = controller.newSearchPhaseResults(mockSearchPhaseContext.getRequest(), numHits); AtomicReference responseRef = new AtomicReference<>(); for (int i = 0; i < numHits; i++) { @@ -254,7 +254,7 @@ public void run() throws IOException { public void testExceptionFailsPhase() throws IOException { MockSearchPhaseContext mockSearchPhaseContext = new MockSearchPhaseContext(2); SearchPhaseController controller = new SearchPhaseController(Settings.EMPTY, BigArrays.NON_RECYCLING_INSTANCE, null); - InitialSearchPhase.SearchPhaseResults results = + InitialSearchPhase.ArraySearchPhaseResults results = controller.newSearchPhaseResults(mockSearchPhaseContext.getRequest(), 2); AtomicReference responseRef = new AtomicReference<>(); int resultSetSize = randomIntBetween(2, 10); @@ -307,7 +307,7 @@ public void run() throws IOException { public void testCleanupIrrelevantContexts() throws IOException { // contexts that are not fetched should be cleaned up MockSearchPhaseContext mockSearchPhaseContext = new MockSearchPhaseContext(2); SearchPhaseController controller = new SearchPhaseController(Settings.EMPTY, BigArrays.NON_RECYCLING_INSTANCE, null); - InitialSearchPhase.SearchPhaseResults results = + InitialSearchPhase.ArraySearchPhaseResults results = controller.newSearchPhaseResults(mockSearchPhaseContext.getRequest(), 2); AtomicReference responseRef = new AtomicReference<>(); int resultSetSize = 1; diff --git a/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java b/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java index b0bf4cc6626fd..f4e45c300e728 100644 --- a/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java @@ -106,7 +106,7 @@ public void onFailure(Exception e) { new TransportSearchAction.SearchTimeProvider(0, 0, () -> 0), 0, null, - new InitialSearchPhase.SearchPhaseResults<>(shardsIter.size())) { + new InitialSearchPhase.ArraySearchPhaseResults<>(shardsIter.size())) { @Override protected void executePhaseOnShard(SearchShardIterator shardIt, ShardRouting shard, @@ -207,7 +207,7 @@ public void sendFreeContext(Transport.Connection connection, long contextId, Ori new TransportSearchAction.SearchTimeProvider(0, 0, () -> 0), 0, null, - new InitialSearchPhase.SearchPhaseResults<>(shardsIter.size())) { + new InitialSearchPhase.ArraySearchPhaseResults<>(shardsIter.size())) { TestSearchResponse response = new TestSearchResponse(); @Override @@ -232,7 +232,7 @@ protected SearchPhase getNextPhase(SearchPhaseResults res @Override public void run() throws IOException { for (int i = 0; i < results.getNumShards(); i++) { - TestSearchPhaseResult result = results.results.get(i); + TestSearchPhaseResult result = results.getAtomicArray().get(i); assertEquals(result.node.getId(), result.getSearchShardTarget().getNodeId()); sendReleaseSearchContext(result.getRequestId(), new MockConnection(result.node), OriginalIndices.NONE); } diff --git a/core/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java b/core/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java index e39bd79defa8e..e6d1e20147b90 100644 --- a/core/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java @@ -288,7 +288,7 @@ public void testConsumer() { SearchRequest request = new SearchRequest(); request.source(new SearchSourceBuilder().aggregation(AggregationBuilders.avg("foo"))); request.setBatchedReduceSize(bufferSize); - InitialSearchPhase.SearchPhaseResults consumer = searchPhaseController.newSearchPhaseResults(request, 3); + InitialSearchPhase.ArraySearchPhaseResults consumer = searchPhaseController.newSearchPhaseResults(request, 3); QuerySearchResult result = new QuerySearchResult(0, new SearchShardTarget("node", new Index("a", "b"), 0, null)); result.topDocs(new TopDocs(0, new ScoreDoc[0], 0.0F), new DocValueFormat[0]); InternalAggregations aggs = new InternalAggregations(Arrays.asList(new InternalMax("test", 1.0D, DocValueFormat.RAW, @@ -335,7 +335,7 @@ public void testConsumerConcurrently() throws InterruptedException { SearchRequest request = new SearchRequest(); request.source(new SearchSourceBuilder().aggregation(AggregationBuilders.avg("foo"))); request.setBatchedReduceSize(bufferSize); - InitialSearchPhase.SearchPhaseResults consumer = + InitialSearchPhase.ArraySearchPhaseResults consumer = searchPhaseController.newSearchPhaseResults(request, expectedNumResults); AtomicInteger max = new AtomicInteger(); Thread[] threads = new Thread[expectedNumResults]; @@ -374,7 +374,7 @@ public void testConsumerOnlyAggs() throws InterruptedException { SearchRequest request = new SearchRequest(); request.source(new SearchSourceBuilder().aggregation(AggregationBuilders.avg("foo")).size(0)); request.setBatchedReduceSize(bufferSize); - InitialSearchPhase.SearchPhaseResults consumer = + InitialSearchPhase.ArraySearchPhaseResults consumer = searchPhaseController.newSearchPhaseResults(request, expectedNumResults); AtomicInteger max = new AtomicInteger(); for (int i = 0; i < expectedNumResults; i++) { @@ -407,7 +407,7 @@ public void testConsumerOnlyHits() throws InterruptedException { request.source(new SearchSourceBuilder().size(randomIntBetween(1, 10))); } request.setBatchedReduceSize(bufferSize); - InitialSearchPhase.SearchPhaseResults consumer = + InitialSearchPhase.ArraySearchPhaseResults consumer = searchPhaseController.newSearchPhaseResults(request, expectedNumResults); AtomicInteger max = new AtomicInteger(); for (int i = 0; i < expectedNumResults; i++) { @@ -450,7 +450,7 @@ public void testNewSearchPhaseResults() { } } request.setBatchedReduceSize(bufferSize); - InitialSearchPhase.SearchPhaseResults consumer + InitialSearchPhase.ArraySearchPhaseResults consumer = searchPhaseController.newSearchPhaseResults(request, expectedNumResults); if ((hasAggs || hasTopDocs) && expectedNumResults > bufferSize) { assertThat("expectedNumResults: " + expectedNumResults + " bufferSize: " + bufferSize, @@ -466,7 +466,7 @@ public void testReduceTopNWithFromOffset() { SearchRequest request = new SearchRequest(); request.source(new SearchSourceBuilder().size(5).from(5)); request.setBatchedReduceSize(randomIntBetween(2, 4)); - InitialSearchPhase.SearchPhaseResults consumer = + InitialSearchPhase.ArraySearchPhaseResults consumer = searchPhaseController.newSearchPhaseResults(request, 4); int score = 100; for (int i = 0; i < 4; i++) { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/msearch.json b/rest-api-spec/src/main/resources/rest-api-spec/api/msearch.json index 69d9145207cba..8b3618953a5ba 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/msearch.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/msearch.json @@ -28,6 +28,11 @@ "typed_keys": { "type" : "boolean", "description" : "Specify whether aggregation and suggester names should be prefixed by their respective types in the response" + }, + "pre_filter_shards_after" : { + "type" : "number", + "description" : "A threshold that enforces a pre-filter roundtrip to prefilter search shards based on query rewriting if the number of shards the search request expands to exceeds the threshold. This filter roundtrip can limit the number of shards significantly if for instance a shard can not match any documents based on it's rewrite method ie. if date filters are mandatory to match but the shard bounds and the query are disjoint.", + "default" : 128 } } }, diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/search.json b/rest-api-spec/src/main/resources/rest-api-spec/api/search.json index 92086d26a2768..e2df23605b4aa 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/search.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/search.json @@ -168,6 +168,11 @@ "type" : "number", "description" : "The number of concurrent shard requests this search executes concurrently. This value should be used to limit the impact of the search on the cluster in order to limit the number of concurrent shard requests", "default" : "The default grows with the number of nodes in the cluster but is at most 256." + }, + "pre_filter_shards_after" : { + "type" : "number", + "description" : "A threshold that enforces a pre-filter roundtrip to prefilter search shards based on query rewriting if the number of shards the search request expands to exceeds the threshold. This filter roundtrip can limit the number of shards significantly if for instance a shard can not match any documents based on it's rewrite method ie. if date filters are mandatory to match but the shard bounds and the query are disjoint.", + "default" : 128 } } }, diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search/140_pre_filter_search_shards.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search/140_pre_filter_search_shards.yml new file mode 100644 index 0000000000000..d82063b8f0325 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search/140_pre_filter_search_shards.yml @@ -0,0 +1,101 @@ +setup: + - do: + indices.create: + index: index_1 + body: + settings: + number_of_shards: 1 + mappings: + doc: + properties: + created_at: + type: date + format: "yyyy-MM-dd" + - do: + indices.create: + index: index_2 + body: + settings: + number_of_shards: 1 + mappings: + doc: + properties: + created_at: + type: date + format: "yyyy-MM-dd" + - do: + indices.create: + index: index_3 + body: + settings: + number_of_shards: 1 + mappings: + doc: + properties: + created_at: + type: date + format: "yyyy-MM-dd" + + +--- +"pre_filter_shards_after with invalid parameter": + - skip: + version: " - 5.99.99" + reason: this was added in 6.0.0 + - do: + catch: /preFilterShardsAfter must be >= 1/ + search: + index: test_1 + pre_filter_shards_after: 0 + +--- +"pre_filter_shards_after with shards that have no hit": + - skip: + version: " - 5.99.99" + reason: this was added in 6.0.0 + - do: + index: + index: index_1 + type: doc + id: 1 + body: { "created_at": "2016-01-01" } + + - do: + index: + index: index_2 + type: doc + id: 2 + body: { "created_at": "2017-01-01" } + + - do: + index: + index: index_3 + type: doc + id: 3 + body: { "created_at": "2018-01-01" } + - do: + indices.refresh: {} + + + - do: + search: + body: { "size" : 0, "query" : { "range" : { "created_at" : { "gte" : "2016-02-01", "lt": "2018-02-01"} } } } + + - match: { _shards.total: 3 } + - match: { _shards.successful: 3 } + - match: { _shards.failed: 0 } + - match: { hits.total: 2 } + + - do: + search: + pre_filter_shards_after: 1 + body: { "size" : 0, "query" : { "range" : { "created_at" : { "gte" : "2016-02-01", "lt": "2018-02-01"} } } } + + - match: { _shards.total: 2 } + - match: { _shards.successful: 2 } + - match: { _shards.failed: 0 } + - match: { hits.total: 2 } + + + + diff --git a/test/framework/src/main/java/org/elasticsearch/test/client/RandomizingClient.java b/test/framework/src/main/java/org/elasticsearch/test/client/RandomizingClient.java index 7076a80f29bbb..3d76396cf01d0 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/client/RandomizingClient.java +++ b/test/framework/src/main/java/org/elasticsearch/test/client/RandomizingClient.java @@ -38,6 +38,7 @@ public class RandomizingClient extends FilterClient { private final String defaultPreference; private final int batchedReduceSize; private final int maxConcurrentShardRequests; + private final int preFilterAfter; public RandomizingClient(Client client, Random random) { @@ -61,6 +62,11 @@ public RandomizingClient(Client client, Random random) { } else { this.maxConcurrentShardRequests = -1; // randomly use the default } + if (random.nextBoolean()) { + preFilterAfter = 1 + random.nextInt(128); + } else { + preFilterAfter = -1; + } } @Override @@ -70,6 +76,9 @@ public SearchRequestBuilder prepareSearch(String... indices) { if (maxConcurrentShardRequests != -1) { searchRequestBuilder.setMaxConcurrentShardRequests(maxConcurrentShardRequests); } + if (preFilterAfter != -1) { + searchRequestBuilder.setPreFilterShardsAfter(preFilterAfter); + } return searchRequestBuilder; } From f784b8e0dd30400d245020830c890fa3b3db3599 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 11 Jul 2017 23:59:53 +0200 Subject: [PATCH 02/12] handle null query --- .../main/java/org/elasticsearch/search/SearchService.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/search/SearchService.java b/core/src/main/java/org/elasticsearch/search/SearchService.java index 8f192bccd1243..00123c78783e4 100644 --- a/core/src/main/java/org/elasticsearch/search/SearchService.java +++ b/core/src/main/java/org/elasticsearch/search/SearchService.java @@ -43,6 +43,7 @@ import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.query.InnerHitContextBuilder; import org.elasticsearch.index.query.MatchNoneQueryBuilder; +import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexShard; @@ -841,7 +842,11 @@ public AliasFilter buildAliasFilter(ClusterState state, String index, String... */ public boolean canMatch(ShardSearchRequest request) throws IOException { try (DefaultSearchContext context = createSearchContext(request, defaultSearchTimeout, null)) { - return context.request().source().query() instanceof MatchNoneQueryBuilder == false; + QueryBuilder queryBuilder = context.request().source().query(); + if (queryBuilder != null) { + return queryBuilder instanceof MatchNoneQueryBuilder == false; + } + return true; // null query means match_all } } } From f05ecfb9bea295d32d1feced3db3f57ed3f3a376 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 12 Jul 2017 00:01:47 +0200 Subject: [PATCH 03/12] trigger more interesting random thresholds --- .../java/org/elasticsearch/test/client/RandomizingClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/framework/src/main/java/org/elasticsearch/test/client/RandomizingClient.java b/test/framework/src/main/java/org/elasticsearch/test/client/RandomizingClient.java index 3d76396cf01d0..3bf89247a539d 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/client/RandomizingClient.java +++ b/test/framework/src/main/java/org/elasticsearch/test/client/RandomizingClient.java @@ -63,7 +63,7 @@ public RandomizingClient(Client client, Random random) { this.maxConcurrentShardRequests = -1; // randomly use the default } if (random.nextBoolean()) { - preFilterAfter = 1 + random.nextInt(128); + preFilterAfter = 1 + random.nextInt(1 << random.nextInt(7)); } else { preFilterAfter = -1; } From 3602ae80d85b186403d274c510f9a523207e658c Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 12 Jul 2017 00:19:02 +0200 Subject: [PATCH 04/12] add checks for global aggs and if source is empty --- .../java/org/elasticsearch/search/SearchService.java | 11 ++++++++--- .../search/aggregations/AggregatorFactories.java | 10 ++++++++-- 2 files changed, 16 insertions(+), 5 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/search/SearchService.java b/core/src/main/java/org/elasticsearch/search/SearchService.java index 00123c78783e4..2637081c43b77 100644 --- a/core/src/main/java/org/elasticsearch/search/SearchService.java +++ b/core/src/main/java/org/elasticsearch/search/SearchService.java @@ -842,9 +842,14 @@ public AliasFilter buildAliasFilter(ClusterState state, String index, String... */ public boolean canMatch(ShardSearchRequest request) throws IOException { try (DefaultSearchContext context = createSearchContext(request, defaultSearchTimeout, null)) { - QueryBuilder queryBuilder = context.request().source().query(); - if (queryBuilder != null) { - return queryBuilder instanceof MatchNoneQueryBuilder == false; + SearchSourceBuilder source = context.request().source(); + if (source != null) { + QueryBuilder queryBuilder = source.query(); + AggregatorFactories.Builder aggregations = source.aggregations(); + boolean hasGlobalAggs = aggregations != null && aggregations.hasGlobalAggregationBuilder(); + if (queryBuilder != null && hasGlobalAggs == false) { // we need to executed hasGlobalAggs is equivalent to match all + return queryBuilder instanceof MatchNoneQueryBuilder == false; + } } return true; // null query means match_all } diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java b/core/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java index eea346b0c3bf6..95b5ebec6b0c6 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java @@ -26,6 +26,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.query.QueryRewriteContext; +import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregationBuilder; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.elasticsearch.search.aggregations.support.AggregationPath; import org.elasticsearch.search.aggregations.support.AggregationPath.PathElement; @@ -282,8 +283,13 @@ public void writeTo(StreamOutput out) throws IOException { } } - public Builder addAggregators(AggregatorFactories factories) { - throw new UnsupportedOperationException("This needs to be removed"); + public boolean hasGlobalAggregationBuilder() { + for (AggregationBuilder builder : aggregationBuilders) { + if (builder instanceof GlobalAggregationBuilder) { + return true; + } + } + return false; } public Builder addAggregator(AggregationBuilder factory) { From 9ade2f38473b7deb84a05bd7edc1889b9cec337b Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 12 Jul 2017 08:34:29 +0200 Subject: [PATCH 05/12] fix prefilter related test failures --- .../action/search/AbstractSearchAsyncAction.java | 1 + .../action/search/CanMatchPreFilterSearchPhase.java | 10 ++++++++++ .../action/search/InitialSearchPhase.java | 2 ++ .../org/elasticsearch/action/search/SearchRequest.java | 2 +- .../action/admin/cluster/node/tasks/TasksIT.java | 3 +++ .../elasticsearch/broadcast/BroadcastActionsIT.java | 2 +- .../java/org/elasticsearch/cluster/NoMasterNodeIT.java | 2 +- .../elasticsearch/indices/IndicesRequestCacheIT.java | 9 ++++++--- 8 files changed, 25 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/core/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index 173da599671a0..4b543b9b642d5 100644 --- a/core/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -167,6 +167,7 @@ private ShardSearchFailure[] buildShardFailures() { } public final void onShardFailure(final int shardIndex, @Nullable SearchShardTarget shardTarget, Exception e) { + results.consumeShardFailure(shardIndex); // we don't aggregate shard failures on non active shards (but do keep the header counts right) if (TransportActions.isShardNotAvailableException(e)) { return; diff --git a/core/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java b/core/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java index 9caf2c4d2cf86..0a4b4f09e15ea 100644 --- a/core/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java +++ b/core/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java @@ -72,6 +72,7 @@ protected void executePhaseOnShard(SearchShardIterator shardIt, ShardRouting sha @Override protected SearchPhase getNextPhase(SearchPhaseResults results, SearchPhaseContext context) { + return phaseFactory.apply(getIterator((BitSetSearchPhaseResults) results, shardsIts)); } @@ -130,6 +131,15 @@ boolean hasResult(int shardIndex) { } } + @Override + void consumeShardFailure(int shardIndex) { + // we have to carry over shard failures in order to account for them in the response. + synchronized (possibleMatches) { + possibleMatches.set(shardIndex); + numMatches++; + } + } + @Override Stream getSuccessfulResults() { return Stream.empty(); diff --git a/core/src/main/java/org/elasticsearch/action/search/InitialSearchPhase.java b/core/src/main/java/org/elasticsearch/action/search/InitialSearchPhase.java index 65892969ac3b3..d5d40ea0a7c2c 100644 --- a/core/src/main/java/org/elasticsearch/action/search/InitialSearchPhase.java +++ b/core/src/main/java/org/elasticsearch/action/search/InitialSearchPhase.java @@ -256,6 +256,8 @@ final int getNumShards() { */ abstract boolean hasResult(int shardIndex); + void consumeShardFailure(int shardIndex) {} + AtomicArray getAtomicArray() { throw new UnsupportedOperationException(); } diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchRequest.java b/core/src/main/java/org/elasticsearch/action/search/SearchRequest.java index 3522d2d7b9776..3347f70e50cc6 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchRequest.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchRequest.java @@ -58,7 +58,7 @@ public final class SearchRequest extends ActionRequest implements IndicesRequest private static final ToXContent.Params FORMAT_PARAMS = new ToXContent.MapParams(Collections.singletonMap("pretty", "false")); - public static final int DEFAULT_PRE_FILTER_SHARDS_AFTER = 128; + public static final int DEFAULT_PRE_FILTER_SHARDS_AFTER = 1; private SearchType searchType = SearchType.DEFAULT; diff --git a/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java b/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java index a2cab6b85abde..343b2d61ec0bb 100644 --- a/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java +++ b/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java @@ -379,6 +379,9 @@ public void testSearchTaskDescriptions() { assertTrue(taskInfo.getDescription(), Regex.simpleMatch("id[*], size[1], lastEmittedDoc[null]", taskInfo.getDescription())); break; + case SearchTransportService.QUERY_CAN_MATCH_NAME: + assertTrue(taskInfo.getDescription(), Regex.simpleMatch("shardId[[test][*]]", taskInfo.getDescription())); + break; default: fail("Unexpected action [" + taskInfo.getAction() + "] with description [" + taskInfo.getDescription() + "]"); } diff --git a/core/src/test/java/org/elasticsearch/broadcast/BroadcastActionsIT.java b/core/src/test/java/org/elasticsearch/broadcast/BroadcastActionsIT.java index 9eb42075edab9..d6f716adce5f6 100644 --- a/core/src/test/java/org/elasticsearch/broadcast/BroadcastActionsIT.java +++ b/core/src/test/java/org/elasticsearch/broadcast/BroadcastActionsIT.java @@ -39,7 +39,7 @@ protected int maximumNumberOfReplicas() { } public void testBroadcastOperations() throws IOException { - assertAcked(prepareCreate("test", 1).execute().actionGet(5000)); + assertAcked(prepareCreate("test", 1)); NumShards numShards = getNumShards("test"); diff --git a/core/src/test/java/org/elasticsearch/cluster/NoMasterNodeIT.java b/core/src/test/java/org/elasticsearch/cluster/NoMasterNodeIT.java index 35d46879639ae..299aff866a1c2 100644 --- a/core/src/test/java/org/elasticsearch/cluster/NoMasterNodeIT.java +++ b/core/src/test/java/org/elasticsearch/cluster/NoMasterNodeIT.java @@ -212,7 +212,7 @@ public void testNoMasterActionsWriteMasterBlock() throws Exception { SearchResponse searchResponse = client().prepareSearch("test1").get(); assertHitCount(searchResponse, 1L); - countResponse = client().prepareSearch("test2").setSize(0).get(); + countResponse = client().prepareSearch("test2").setSize(0).setPreFilterShardsAfter(Integer.MAX_VALUE).get(); assertThat(countResponse.getTotalShards(), equalTo(2)); assertThat(countResponse.getSuccessfulShards(), equalTo(1)); diff --git a/core/src/test/java/org/elasticsearch/indices/IndicesRequestCacheIT.java b/core/src/test/java/org/elasticsearch/indices/IndicesRequestCacheIT.java index 7f8fe5d40a9f2..e8060b4abf2c9 100644 --- a/core/src/test/java/org/elasticsearch/indices/IndicesRequestCacheIT.java +++ b/core/src/test/java/org/elasticsearch/indices/IndicesRequestCacheIT.java @@ -111,7 +111,8 @@ public void testQueryRewrite() throws Exception { equalTo(0L)); final SearchResponse r1 = client().prepareSearch("index").setSearchType(SearchType.QUERY_THEN_FETCH).setSize(0) - .setQuery(QueryBuilders.rangeQuery("s").gte("2016-03-19").lte("2016-03-25")).get(); + .setQuery(QueryBuilders.rangeQuery("s").gte("2016-03-19").lte("2016-03-25")).setPreFilterShardsAfter(Integer.MAX_VALUE) + .get(); assertSearchResponse(r1); assertThat(r1.getHits().getTotalHits(), equalTo(7L)); assertThat(client().admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), @@ -120,7 +121,8 @@ public void testQueryRewrite() throws Exception { equalTo(5L)); final SearchResponse r2 = client().prepareSearch("index").setSearchType(SearchType.QUERY_THEN_FETCH).setSize(0) - .setQuery(QueryBuilders.rangeQuery("s").gte("2016-03-20").lte("2016-03-26")).get(); + .setQuery(QueryBuilders.rangeQuery("s").gte("2016-03-20").lte("2016-03-26")) + .setPreFilterShardsAfter(Integer.MAX_VALUE).get(); assertSearchResponse(r2); assertThat(r2.getHits().getTotalHits(), equalTo(7L)); assertThat(client().admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), @@ -129,7 +131,8 @@ public void testQueryRewrite() throws Exception { equalTo(7L)); final SearchResponse r3 = client().prepareSearch("index").setSearchType(SearchType.QUERY_THEN_FETCH).setSize(0) - .setQuery(QueryBuilders.rangeQuery("s").gte("2016-03-21").lte("2016-03-27")).get(); + .setQuery(QueryBuilders.rangeQuery("s").gte("2016-03-21").lte("2016-03-27")).setPreFilterShardsAfter(Integer.MAX_VALUE) + .get(); assertSearchResponse(r3); assertThat(r3.getHits().getTotalHits(), equalTo(7L)); assertThat(client().admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), From cd68719af315a286afbe317ae8531a324b8887e8 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 12 Jul 2017 11:19:11 +0200 Subject: [PATCH 06/12] apply feedback --- .../search/TransportNoopSearchAction.java | 2 +- .../client/RestHighLevelClientTests.java | 2 +- .../search/AbstractSearchAsyncAction.java | 59 +++++++++++-------- .../search/CanMatchPreFilterSearchPhase.java | 55 ++++++++--------- .../action/search/FetchSearchPhase.java | 2 +- .../action/search/InitialSearchPhase.java | 35 +++++++++-- .../action/search/SearchRequest.java | 2 +- .../action/search/SearchResponse.java | 31 ++++++++-- .../search/SearchScrollAsyncAction.java | 2 +- .../action/search/SearchShardIterator.java | 10 ++++ .../action/search/TransportSearchAction.java | 19 +++++- .../rest/action/RestActions.java | 6 +- .../rest/action/document/RestCountAction.java | 2 +- .../action/search/RestMultiSearchAction.java | 3 +- .../action/search/MockSearchPhaseContext.java | 2 +- .../action/search/SearchResponseTests.java | 13 ++-- .../elasticsearch/cluster/NoMasterNodeIT.java | 2 +- .../reindex/AsyncBulkByScrollActionTests.java | 2 +- .../search/140_pre_filter_search_shards.yml | 31 +++++++++- 19 files changed, 192 insertions(+), 88 deletions(-) diff --git a/client/client-benchmark-noop-api-plugin/src/main/java/org/elasticsearch/plugin/noop/action/search/TransportNoopSearchAction.java b/client/client-benchmark-noop-api-plugin/src/main/java/org/elasticsearch/plugin/noop/action/search/TransportNoopSearchAction.java index 77e7cdab93707..280e0b08f2c72 100644 --- a/client/client-benchmark-noop-api-plugin/src/main/java/org/elasticsearch/plugin/noop/action/search/TransportNoopSearchAction.java +++ b/client/client-benchmark-noop-api-plugin/src/main/java/org/elasticsearch/plugin/noop/action/search/TransportNoopSearchAction.java @@ -53,6 +53,6 @@ protected void doExecute(SearchRequest request, ActionListener l new SearchHit[0], 0L, 0.0f), new InternalAggregations(Collections.emptyList()), new Suggest(Collections.emptyList()), - new SearchProfileShardResults(Collections.emptyMap()), false, false, 1), "", 1, 1, 0, new ShardSearchFailure[0])); + new SearchProfileShardResults(Collections.emptyMap()), false, false, 1), "", 1, 1, 0, 0, new ShardSearchFailure[0])); } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/RestHighLevelClientTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/RestHighLevelClientTests.java index bbc973e231588..c4b3aaf18a489 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/RestHighLevelClientTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/RestHighLevelClientTests.java @@ -153,7 +153,7 @@ public void testInfo() throws IOException { public void testSearchScroll() throws IOException { Header[] headers = randomHeaders(random(), "Header"); SearchResponse mockSearchResponse = new SearchResponse(new SearchResponseSections(SearchHits.empty(), InternalAggregations.EMPTY, - null, false, false, null, 1), randomAlphaOfLengthBetween(5, 10), 5, 5, 100, new ShardSearchFailure[0]); + null, false, false, null, 1), randomAlphaOfLengthBetween(5, 10), 5, 5, 0, 100, new ShardSearchFailure[0]); mockResponse(mockSearchResponse); SearchResponse searchResponse = restHighLevelClient.searchScroll(new SearchScrollRequest(randomAlphaOfLengthBetween(5, 10)), headers); diff --git a/core/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/core/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index 4b543b9b642d5..bd721adaf127a 100644 --- a/core/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -66,6 +66,7 @@ abstract class AbstractSearchAsyncAction exten private final SetOnce> shardFailures = new SetOnce<>(); private final Object shardFailuresMutex = new Object(); private final AtomicInteger successfulOps = new AtomicInteger(); + private final AtomicInteger skippedOps = new AtomicInteger(); private final TransportSearchAction.SearchTimeProvider timeProvider; @@ -106,7 +107,7 @@ public final void start() { if (getNumShards() == 0) { //no search shards to search on, bail with empty response //(it happens with search across _all with no indices around and consistent with broadcast operations) - listener.onResponse(new SearchResponse(InternalSearchResponse.empty(), null, 0, 0, buildTookInMillis(), + listener.onResponse(new SearchResponse(InternalSearchResponse.empty(), null, 0, 0, 0, buildTookInMillis(), ShardSearchFailure.EMPTY_ARRAY)); return; } @@ -167,37 +168,36 @@ private ShardSearchFailure[] buildShardFailures() { } public final void onShardFailure(final int shardIndex, @Nullable SearchShardTarget shardTarget, Exception e) { - results.consumeShardFailure(shardIndex); // we don't aggregate shard failures on non active shards (but do keep the header counts right) - if (TransportActions.isShardNotAvailableException(e)) { - return; - } - AtomicArray shardFailures = this.shardFailures.get(); - // lazily create shard failures, so we can early build the empty shard failure list in most cases (no failures) - if (shardFailures == null) { // this is double checked locking but it's fine since SetOnce uses a volatile read internally - synchronized (shardFailuresMutex) { - shardFailures = this.shardFailures.get(); // read again otherwise somebody else has created it? - if (shardFailures == null) { // still null so we are the first and create a new instance - shardFailures = new AtomicArray<>(getNumShards()); - this.shardFailures.set(shardFailures); + if (TransportActions.isShardNotAvailableException(e) == false) { + AtomicArray shardFailures = this.shardFailures.get(); + // lazily create shard failures, so we can early build the empty shard failure list in most cases (no failures) + if (shardFailures == null) { // this is double checked locking but it's fine since SetOnce uses a volatile read internally + synchronized (shardFailuresMutex) { + shardFailures = this.shardFailures.get(); // read again otherwise somebody else has created it? + if (shardFailures == null) { // still null so we are the first and create a new instance + shardFailures = new AtomicArray<>(getNumShards()); + this.shardFailures.set(shardFailures); + } } } - } - ShardSearchFailure failure = shardFailures.get(shardIndex); - if (failure == null) { - shardFailures.set(shardIndex, new ShardSearchFailure(e, shardTarget)); - } else { - // the failure is already present, try and not override it with an exception that is less meaningless - // for example, getting illegal shard state - if (TransportActions.isReadOverrideException(e)) { + ShardSearchFailure failure = shardFailures.get(shardIndex); + if (failure == null) { shardFailures.set(shardIndex, new ShardSearchFailure(e, shardTarget)); + } else { + // the failure is already present, try and not override it with an exception that is less meaningless + // for example, getting illegal shard state + if (TransportActions.isReadOverrideException(e)) { + shardFailures.set(shardIndex, new ShardSearchFailure(e, shardTarget)); + } } - } - if (results.hasResult(shardIndex)) { - assert failure == null : "shard failed before but shouldn't: " + failure; - successfulOps.decrementAndGet(); // if this shard was successful before (initial phase) we have to adjust the counter + if (results.hasResult(shardIndex)) { + assert failure == null : "shard failed before but shouldn't: " + failure; + successfulOps.decrementAndGet(); // if this shard was successful before (initial phase) we have to adjust the counter + } } + results.consumeShardFailure(shardIndex); } /** @@ -264,7 +264,7 @@ public final SearchRequest getRequest() { @Override public final SearchResponse buildSearchResponse(InternalSearchResponse internalSearchResponse, String scrollId) { return new SearchResponse(internalSearchResponse, scrollId, getNumShards(), successfulOps.get(), - buildTookInMillis(), buildShardFailures()); + skippedOps.get(), buildTookInMillis(), buildShardFailures()); } @Override @@ -313,4 +313,11 @@ public final ShardSearchTransportRequest buildShardSearchRequest(SearchShardIter * @param context the search context for the next phase */ protected abstract SearchPhase getNextPhase(SearchPhaseResults results, SearchPhaseContext context); + + @Override + protected void skipShard(SearchShardIterator iterator) { + super.skipShard(iterator); + successfulOps.incrementAndGet(); + skippedOps.incrementAndGet(); + } } diff --git a/core/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java b/core/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java index 0a4b4f09e15ea..ea5cf831859de 100644 --- a/core/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java +++ b/core/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java @@ -78,36 +78,29 @@ protected SearchPhase getNextPhase(SearchPhaseResults getIterator(BitSetSearchPhaseResults results, GroupShardsIterator shardsIts) { - if (results.numMatches == shardsIts.size()) { - shardsIts.iterator().forEachRemaining(i -> i.reset()); - return shardsIts; - } else if (results.numMatches == 0) { + int cardinality = results.getNumPossibleMatches(); + FixedBitSet possibleMatches = results.getPossibleMatches(); + if (cardinality == 0) { // this is a special case where we have no hit but we need to get at least one search response in order - // to produce a valid search result with all the aggs etc. at least that is what I think is the case... and clint does so - // too :D - Iterator iterator = shardsIts.iterator(); - if (iterator.hasNext()) { - SearchShardIterator next = iterator.next(); - next.reset(); - return new GroupShardsIterator<>(Collections.singletonList(next)); - } + // to produce a valid search result with all the aggs etc. + possibleMatches.set(0); } - List shardIterators = new ArrayList<>(results.numMatches); int i = 0; for (SearchShardIterator iter : shardsIts) { - if (results.possibleMatches.get(i++)) { + if (possibleMatches.get(i++)) { iter.reset(); - shardIterators.add(iter); + } else { + iter.resetAndSkip(); } } - return new GroupShardsIterator<>(shardIterators); + return shardsIts; } private static final class BitSetSearchPhaseResults extends InitialSearchPhase. SearchPhaseResults { - private FixedBitSet possibleMatches; - private int numMatches; + private final FixedBitSet possibleMatches; + private int numPossibleMatches; BitSetSearchPhaseResults(int size) { super(size); @@ -117,27 +110,29 @@ private static final class BitSetSearchPhaseResults extends InitialSearchPhase. @Override void consumeResult(SearchTransportService.CanMatchResponse result) { if (result.canMatch()) { - synchronized (possibleMatches) { - possibleMatches.set(result.getShardIndex()); - numMatches++; - } + consumeShardFailure(result.getShardIndex()); } } @Override boolean hasResult(int shardIndex) { - synchronized (possibleMatches) { - return possibleMatches.get(shardIndex); - } + return false; // unneeded } @Override - void consumeShardFailure(int shardIndex) { + synchronized void consumeShardFailure(int shardIndex) { // we have to carry over shard failures in order to account for them in the response. - synchronized (possibleMatches) { - possibleMatches.set(shardIndex); - numMatches++; - } + possibleMatches.set(shardIndex); + numPossibleMatches++; + } + + + synchronized int getNumPossibleMatches() { + return numPossibleMatches; + } + + synchronized FixedBitSet getPossibleMatches() { + return possibleMatches; } @Override diff --git a/core/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java b/core/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java index e5fdf2eff725e..4712496bc37ec 100644 --- a/core/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java +++ b/core/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java @@ -105,7 +105,7 @@ private void innerRun() throws IOException { -> moveToNextPhase(searchPhaseController, scrollId, reducedQueryPhase, queryAndFetchOptimization ? queryResults : fetchResults); if (queryAndFetchOptimization) { - assert phaseResults.isEmpty() || phaseResults.get(0).fetchResult() != null : "phaseResults emtpy [" + phaseResults.isEmpty() + assert phaseResults.isEmpty() || phaseResults.get(0).fetchResult() != null : "phaseResults empty [" + phaseResults.isEmpty() + "], single result: " + phaseResults.get(0).fetchResult(); // query AND fetch optimization finishPhase.run(); diff --git a/core/src/main/java/org/elasticsearch/action/search/InitialSearchPhase.java b/core/src/main/java/org/elasticsearch/action/search/InitialSearchPhase.java index d5d40ea0a7c2c..fcee980379bf1 100644 --- a/core/src/main/java/org/elasticsearch/action/search/InitialSearchPhase.java +++ b/core/src/main/java/org/elasticsearch/action/search/InitialSearchPhase.java @@ -24,7 +24,6 @@ import org.elasticsearch.action.NoShardAvailableActionException; import org.elasticsearch.action.support.TransportActions; import org.elasticsearch.cluster.routing.GroupShardsIterator; -import org.elasticsearch.cluster.routing.ShardIterator; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.util.concurrent.AtomicArray; @@ -132,7 +131,11 @@ public final void run() throws IOException { assert success; for (int i = 0; i < maxConcurrentShardRequests; i++) { SearchShardIterator shardRoutings = shardsIts.get(i); - performPhaseOnShard(i, shardRoutings, shardRoutings.nextOrNull()); + if (shardRoutings.skip()) { + skipShard(shardRoutings); + } else { + performPhaseOnShard(i, shardRoutings, shardRoutings.nextOrNull()); + } } } @@ -140,7 +143,11 @@ private void maybeExecuteNext() { final int index = shardExecutionIndex.getAndIncrement(); if (index < shardsIts.size()) { SearchShardIterator shardRoutings = shardsIts.get(index); - performPhaseOnShard(index, shardRoutings, shardRoutings.nextOrNull()); + if (shardRoutings.skip()) { + skipShard(shardRoutings); + } else { + performPhaseOnShard(index, shardRoutings, shardRoutings.nextOrNull()); + } } } @@ -171,8 +178,7 @@ public void onFailure(Exception t) { } } - private void onShardResult(FirstResult result, ShardIterator shardIt) { - maybeExecuteNext(); + private void onShardResult(FirstResult result, SearchShardIterator shardIt) { assert result.getShardIndex() != -1 : "shard index is not set"; assert result.getSearchShardTarget() != null : "search shard target must not be null"; onShardSuccess(result); @@ -181,12 +187,24 @@ private void onShardResult(FirstResult result, ShardIterator shardIt) { // cause the successor to read a wrong value from successfulOps if second phase is very fast ie. count etc. // increment all the "future" shards to update the total ops since we some may work and some may not... // and when that happens, we break on total ops, so we must maintain them - final int xTotalOps = totalOps.addAndGet(shardIt.remaining() + 1); + successfulShardExecution(shardIt); + } + + private void successfulShardExecution(SearchShardIterator shardsIt) { + final int remainingOpsOnIterator; + if (shardsIt.skip()) { + remainingOpsOnIterator = shardsIt.remaining(); + } else { + remainingOpsOnIterator = shardsIt.remaining() + 1; + } + final int xTotalOps = totalOps.addAndGet(remainingOpsOnIterator); if (xTotalOps == expectedTotalOps) { onPhaseDone(); } else if (xTotalOps > expectedTotalOps) { throw new AssertionError("unexpected higher total ops [" + xTotalOps + "] compared to expected [" + expectedTotalOps + "]"); + } else { + maybeExecuteNext(); } } @@ -300,4 +318,9 @@ AtomicArray getAtomicArray() { } } + protected void skipShard(SearchShardIterator iterator) { + assert iterator.skip(); + successfulShardExecution(iterator); + } + } diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchRequest.java b/core/src/main/java/org/elasticsearch/action/search/SearchRequest.java index 3347f70e50cc6..3522d2d7b9776 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchRequest.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchRequest.java @@ -58,7 +58,7 @@ public final class SearchRequest extends ActionRequest implements IndicesRequest private static final ToXContent.Params FORMAT_PARAMS = new ToXContent.MapParams(Collections.singletonMap("pretty", "false")); - public static final int DEFAULT_PRE_FILTER_SHARDS_AFTER = 1; + public static final int DEFAULT_PRE_FILTER_SHARDS_AFTER = 128; private SearchType searchType = SearchType.DEFAULT; diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchResponse.java b/core/src/main/java/org/elasticsearch/action/search/SearchResponse.java index 3aa5e3a2adbc6..a1709c469d5f2 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchResponse.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchResponse.java @@ -19,6 +19,7 @@ package org.elasticsearch.action.search; +import org.elasticsearch.Version; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.ParseField; @@ -66,6 +67,8 @@ public class SearchResponse extends ActionResponse implements StatusToXContentOb private int successfulShards; + private int skippedShards; + private ShardSearchFailure[] shardFailures; private long tookInMillis; @@ -74,13 +77,15 @@ public SearchResponse() { } public SearchResponse(SearchResponseSections internalResponse, String scrollId, int totalShards, int successfulShards, - long tookInMillis, ShardSearchFailure[] shardFailures) { + int skippedShards, long tookInMillis, ShardSearchFailure[] shardFailures) { this.internalResponse = internalResponse; this.scrollId = scrollId; this.totalShards = totalShards; this.successfulShards = successfulShards; + this.skippedShards = skippedShards; this.tookInMillis = tookInMillis; this.shardFailures = shardFailures; + assert skippedShards <= totalShards : "skipped: " + skippedShards + " total: " + totalShards; } @Override @@ -147,6 +152,14 @@ public int getSuccessfulShards() { return successfulShards; } + + /** + * The number of shards skipped due to pre-filtering + */ + public int getSkippedShards() { + return skippedShards; + } + /** * The failed number of shards the search was executed on. */ @@ -206,8 +219,8 @@ public XContentBuilder innerToXContent(XContentBuilder builder, Params params) t if (getNumReducePhases() != 1) { builder.field(NUM_REDUCE_PHASES.getPreferredName(), getNumReducePhases()); } - RestActions.buildBroadcastShardsHeader(builder, params, getTotalShards(), getSuccessfulShards(), getFailedShards(), - getShardFailures()); + RestActions.buildBroadcastShardsHeader(builder, params, getTotalShards(), getSuccessfulShards(), getSkippedShards(), + getFailedShards(), getShardFailures()); internalResponse.toXContent(builder, params); return builder; } @@ -226,6 +239,7 @@ public static SearchResponse fromXContent(XContentParser parser) throws IOExcept long tookInMillis = -1; int successfulShards = -1; int totalShards = -1; + int skippedShards = 0; // 0 for BWC String scrollId = null; List failures = new ArrayList<>(); while((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { @@ -265,6 +279,8 @@ public static SearchResponse fromXContent(XContentParser parser) throws IOExcept successfulShards = parser.intValue(); } else if (RestActions.TOTAL_FIELD.match(currentFieldName)) { totalShards = parser.intValue(); + } else if (RestActions.SKIPPED_FIELD.match(currentFieldName)) { + skippedShards = parser.intValue(); } else { parser.skipChildren(); } @@ -287,7 +303,7 @@ public static SearchResponse fromXContent(XContentParser parser) throws IOExcept } SearchResponseSections searchResponseSections = new SearchResponseSections(hits, aggs, suggest, timedOut, terminatedEarly, profile, numReducePhases); - return new SearchResponse(searchResponseSections, scrollId, totalShards, successfulShards, tookInMillis, + return new SearchResponse(searchResponseSections, scrollId, totalShards, successfulShards, skippedShards, tookInMillis, failures.toArray(new ShardSearchFailure[failures.size()])); } @@ -308,6 +324,9 @@ public void readFrom(StreamInput in) throws IOException { } scrollId = in.readOptionalString(); tookInMillis = in.readVLong(); + if (in.getVersion().onOrAfter(Version.V_6_0_0_beta1)) { + skippedShards = in.readVInt(); + } } @Override @@ -324,10 +343,14 @@ public void writeTo(StreamOutput out) throws IOException { out.writeOptionalString(scrollId); out.writeVLong(tookInMillis); + if(out.getVersion().onOrAfter(Version.V_6_0_0_beta1)) { + out.writeVInt(skippedShards); + } } @Override public String toString() { return Strings.toString(this); } + } diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchScrollAsyncAction.java b/core/src/main/java/org/elasticsearch/action/search/SearchScrollAsyncAction.java index aa757a039b899..109a1f30ffc8e 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchScrollAsyncAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchScrollAsyncAction.java @@ -249,7 +249,7 @@ protected final void sendResponse(SearchPhaseController.ReducedQueryPhase queryP scrollId = request.scrollId(); } listener.onResponse(new SearchResponse(internalResponse, scrollId, this.scrollId.getContext().length, successfulOps.get(), - buildTookInMillis(), buildShardFailures())); + 0, buildTookInMillis(), buildShardFailures())); } catch (Exception e) { listener.onFailure(new ReduceSearchPhaseException("fetch", "inner finish failed", e, buildShardFailures())); } diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchShardIterator.java b/core/src/main/java/org/elasticsearch/action/search/SearchShardIterator.java index d3d707771b8db..523faa3cb8d06 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchShardIterator.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchShardIterator.java @@ -34,6 +34,7 @@ public final class SearchShardIterator extends PlainShardIterator { private final OriginalIndices originalIndices; private String clusterAlias; + private boolean skip = false; /** * Creates a {@link PlainShardIterator} instance that iterates over a subset of the given shards @@ -58,4 +59,13 @@ public OriginalIndices getOriginalIndices() { public String getClusterAlias() { return clusterAlias; } + + void resetAndSkip() { + reset(); + skip = true; + } + + boolean skip() { + return skip; + } } diff --git a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index 89c643071341e..71a23fd266253 100644 --- a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -314,13 +314,26 @@ private void executeSearch(SearchTask task, SearchTimeProvider timeProvider, Sea searchRequest.setMaxConcurrentShardRequests(Math.min(256, nodeCount * IndexMetaData.INDEX_NUMBER_OF_SHARDS_SETTING.getDefault(Settings.EMPTY))); } - boolean preFilterSearchShards = searchRequest.searchType() == QUERY_THEN_FETCH && // we can't do this for DFS it needs to fan out - // to all shards all the time - searchRequest.getPreFilterShardsAfter() < shardIterators.size(); + boolean preFilterSearchShards = shouldPreFilterSearchShards(searchRequest, shardIterators); searchAsyncAction(task, searchRequest, shardIterators, timeProvider, connectionLookup, clusterState.version(), Collections.unmodifiableMap(aliasFilter), concreteIndexBoosts, listener, preFilterSearchShards).start(); } + private boolean shouldPreFilterSearchShards(SearchRequest searchRequest, GroupShardsIterator shardIterators) { + SearchSourceBuilder source = searchRequest.source(); + if (source == null) { + return false; + } else if (source.aggregations() != null && source.aggregations().hasGlobalAggregationBuilder()) { + return false; + } else if (source.query() == null) { + // match all in this case + return false; + } + return searchRequest.searchType() == QUERY_THEN_FETCH && // we can't do this for DFS it needs to fan out + // to all shards all the time + searchRequest.getPreFilterShardsAfter() < shardIterators.size(); + } + static GroupShardsIterator mergeShardsIterators(GroupShardsIterator localShardsIterator, OriginalIndices localIndices, List remoteShardIterators) { diff --git a/core/src/main/java/org/elasticsearch/rest/action/RestActions.java b/core/src/main/java/org/elasticsearch/rest/action/RestActions.java index 295ae7418c65d..5a77a1042008b 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/RestActions.java +++ b/core/src/main/java/org/elasticsearch/rest/action/RestActions.java @@ -52,6 +52,7 @@ public class RestActions { public static final ParseField _SHARDS_FIELD = new ParseField("_shards"); public static final ParseField TOTAL_FIELD = new ParseField("total"); public static final ParseField SUCCESSFUL_FIELD = new ParseField("successful"); + public static final ParseField SKIPPED_FIELD = new ParseField("skipped"); public static final ParseField FAILED_FIELD = new ParseField("failed"); public static final ParseField FAILURES_FIELD = new ParseField("failures"); @@ -73,16 +74,17 @@ public static long parseVersion(RestRequest request, long defaultVersion) { public static void buildBroadcastShardsHeader(XContentBuilder builder, Params params, BroadcastResponse response) throws IOException { buildBroadcastShardsHeader(builder, params, - response.getTotalShards(), response.getSuccessfulShards(), response.getFailedShards(), + response.getTotalShards(), response.getSuccessfulShards(), 0, response.getFailedShards(), response.getShardFailures()); } public static void buildBroadcastShardsHeader(XContentBuilder builder, Params params, - int total, int successful, int failed, + int total, int successful, int skipped, int failed, ShardOperationFailedException[] shardFailures) throws IOException { builder.startObject(_SHARDS_FIELD.getPreferredName()); builder.field(TOTAL_FIELD.getPreferredName(), total); builder.field(SUCCESSFUL_FIELD.getPreferredName(), successful); + builder.field(SKIPPED_FIELD.getPreferredName(), skipped); builder.field(FAILED_FIELD.getPreferredName(), failed); if (shardFailures != null && shardFailures.length > 0) { builder.startArray(FAILURES_FIELD.getPreferredName()); diff --git a/core/src/main/java/org/elasticsearch/rest/action/document/RestCountAction.java b/core/src/main/java/org/elasticsearch/rest/action/document/RestCountAction.java index ed9a31aebd537..a66b17cc5a5e5 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/document/RestCountAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/document/RestCountAction.java @@ -98,7 +98,7 @@ public RestResponse buildResponse(SearchResponse response, XContentBuilder build } builder.field("count", response.getHits().getTotalHits()); buildBroadcastShardsHeader(builder, request, response.getTotalShards(), response.getSuccessfulShards(), - response.getFailedShards(), response.getShardFailures()); + 0, response.getFailedShards(), response.getShardFailures()); builder.endObject(); return new BytesRestResponse(response.status(), builder); diff --git a/core/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java b/core/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java index a18cb5f266402..267278de4f930 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java @@ -103,7 +103,8 @@ public static MultiSearchRequest parseRequest(RestRequest restRequest, boolean a List requests = multiRequest.requests(); preFilterShardsAfter = Math.max(1, preFilterShardsAfter / (requests.size()+1)); for (SearchRequest request : requests) { - request.setPreFilterSearchShardsAfter(preFilterShardsAfter); + // preserve if it's set on the request + request.setPreFilterSearchShardsAfter(Math.min(preFilterShardsAfter, request.getPreFilterShardsAfter())); } return multiRequest; } diff --git a/core/src/test/java/org/elasticsearch/action/search/MockSearchPhaseContext.java b/core/src/test/java/org/elasticsearch/action/search/MockSearchPhaseContext.java index 9135778479e2e..2f06dd873215e 100644 --- a/core/src/test/java/org/elasticsearch/action/search/MockSearchPhaseContext.java +++ b/core/src/test/java/org/elasticsearch/action/search/MockSearchPhaseContext.java @@ -83,7 +83,7 @@ public SearchRequest getRequest() { @Override public SearchResponse buildSearchResponse(InternalSearchResponse internalSearchResponse, String scrollId) { - return new SearchResponse(internalSearchResponse, scrollId, numShards, numSuccess.get(), 0, + return new SearchResponse(internalSearchResponse, scrollId, numShards, numSuccess.get(), 0, 0, failures.toArray(new ShardSearchFailure[0])); } diff --git a/core/src/test/java/org/elasticsearch/action/search/SearchResponseTests.java b/core/src/test/java/org/elasticsearch/action/search/SearchResponseTests.java index c91fd7377a5fc..02c4964af3cc5 100644 --- a/core/src/test/java/org/elasticsearch/action/search/SearchResponseTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/SearchResponseTests.java @@ -98,8 +98,9 @@ private SearchResponse createTestItem(boolean minimal, ShardSearchFailure... sha Boolean terminatedEarly = randomBoolean() ? null : randomBoolean(); int numReducePhases = randomIntBetween(1, 10); long tookInMillis = randomNonNegativeLong(); - int successfulShards = randomInt(); - int totalShards = randomInt(); + int totalShards = randomIntBetween(1, Integer.MAX_VALUE); + int successfulShards = randomIntBetween(0, totalShards); + int skippedShards = randomIntBetween(0, totalShards); InternalSearchResponse internalSearchResponse; if (minimal == false) { SearchHits hits = SearchHitsTests.createTestItem(); @@ -111,7 +112,8 @@ private SearchResponse createTestItem(boolean minimal, ShardSearchFailure... sha } else { internalSearchResponse = InternalSearchResponse.empty(); } - return new SearchResponse(internalSearchResponse, null, totalShards, successfulShards, tookInMillis, shardSearchFailures); + return new SearchResponse(internalSearchResponse, null, totalShards, successfulShards, skippedShards, tookInMillis, + shardSearchFailures); } /** @@ -192,7 +194,7 @@ public void testToXContent() { hit.score(2.0f); SearchHit[] hits = new SearchHit[] { hit }; SearchResponse response = new SearchResponse( - new InternalSearchResponse(new SearchHits(hits, 100, 1.5f), null, null, null, false, null, 1), null, 0, 0, 0, + new InternalSearchResponse(new SearchHits(hits, 100, 1.5f), null, null, null, false, null, 1), null, 0, 0, 0, 0, new ShardSearchFailure[0]); StringBuilder expectedString = new StringBuilder(); expectedString.append("{"); @@ -203,6 +205,7 @@ public void testToXContent() { { expectedString.append("{\"total\":0,"); expectedString.append("\"successful\":0,"); + expectedString.append("\"skipped\":0,"); expectedString.append("\"failed\":0},"); } expectedString.append("\"hits\":"); @@ -215,4 +218,4 @@ public void testToXContent() { expectedString.append("}"); assertEquals(expectedString.toString(), Strings.toString(response)); } -} \ No newline at end of file +} diff --git a/core/src/test/java/org/elasticsearch/cluster/NoMasterNodeIT.java b/core/src/test/java/org/elasticsearch/cluster/NoMasterNodeIT.java index 299aff866a1c2..35d46879639ae 100644 --- a/core/src/test/java/org/elasticsearch/cluster/NoMasterNodeIT.java +++ b/core/src/test/java/org/elasticsearch/cluster/NoMasterNodeIT.java @@ -212,7 +212,7 @@ public void testNoMasterActionsWriteMasterBlock() throws Exception { SearchResponse searchResponse = client().prepareSearch("test1").get(); assertHitCount(searchResponse, 1L); - countResponse = client().prepareSearch("test2").setSize(0).setPreFilterShardsAfter(Integer.MAX_VALUE).get(); + countResponse = client().prepareSearch("test2").setSize(0).get(); assertThat(countResponse.getTotalShards(), equalTo(2)); assertThat(countResponse.getSuccessfulShards(), equalTo(1)); diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/AsyncBulkByScrollActionTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/AsyncBulkByScrollActionTests.java index 315621bf86f42..7f59987617f2b 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/AsyncBulkByScrollActionTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/AsyncBulkByScrollActionTests.java @@ -452,7 +452,7 @@ public ScheduledFuture schedule(TimeValue delay, String name, Runnable comman SearchHit hit = new SearchHit(0, "id", new Text("type"), emptyMap()); SearchHits hits = new SearchHits(new SearchHit[] { hit }, 0, 0); InternalSearchResponse internalResponse = new InternalSearchResponse(hits, null, null, null, false, false, 1); - SearchResponse searchResponse = new SearchResponse(internalResponse, scrollId(), 5, 4, randomLong(), null); + SearchResponse searchResponse = new SearchResponse(internalResponse, scrollId(), 5, 4, 0, randomLong(), null); if (randomBoolean()) { client.lastScroll.get().listener.onResponse(searchResponse); diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search/140_pre_filter_search_shards.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search/140_pre_filter_search_shards.yml index d82063b8f0325..14ffec3c5135a 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search/140_pre_filter_search_shards.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search/140_pre_filter_search_shards.yml @@ -83,19 +83,46 @@ setup: - match: { _shards.total: 3 } - match: { _shards.successful: 3 } + - match: { _shards.skipped: 0 } - match: { _shards.failed: 0 } - match: { hits.total: 2 } + # this is the case where we have an empty body and don't skip anything since it's match_all + - do: + search: + pre_filter_shards_after: 1 + + - match: { _shards.total: 3 } + - match: { _shards.successful: 3 } + - match: { _shards.skipped: 0 } + - match: { _shards.failed: 0 } + - match: { hits.total: 3 } + + # this is a case where we can actually skip due to rewrite - do: search: pre_filter_shards_after: 1 body: { "size" : 0, "query" : { "range" : { "created_at" : { "gte" : "2016-02-01", "lt": "2018-02-01"} } } } - - match: { _shards.total: 2 } - - match: { _shards.successful: 2 } + - match: { _shards.total: 3 } + - match: { _shards.successful: 3 } + - match: { _shards.skipped : 1} - match: { _shards.failed: 0 } - match: { hits.total: 2 } + # this case we skip all except of one since we need a real result + - do: + search: + pre_filter_shards_after: 1 + body: { "size" : 0, "query" : { "range" : { "created_at" : { "gte" : "2019-02-01", "lt": "2020-02-01"} } } } + + - match: { _shards.total: 3 } + - match: { _shards.successful: 3 } + - match: { _shards.skipped : 2} # skip 2 and execute one to fetch the actual empty result + - match: { _shards.failed: 0 } + - match: { hits.total: 0 } + + From 85be0b0eff0ce60b141e48be80a9fa8422afa0e5 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 12 Jul 2017 15:26:57 +0200 Subject: [PATCH 07/12] fix tests --- .../action/search/TransportSearchAction.java | 9 +---- .../rest/action/RestActions.java | 6 ++- .../elasticsearch/search/SearchService.java | 23 ++++++++--- .../aggregations/AggregatorFactories.java | 12 +++++- .../bucket/children-aggregation.asciidoc | 1 + .../pattern-replace-charfilter.asciidoc | 1 + .../tokenizers/edgengram-tokenizer.asciidoc | 1 + docs/reference/getting-started.asciidoc | 3 ++ docs/reference/how-to/recipes.asciidoc | 3 ++ .../mapping/params/normalizer.asciidoc | 2 + docs/reference/mapping/types/range.asciidoc | 1 + .../query-dsl/percolate-query.asciidoc | 2 + docs/reference/search/count.asciidoc | 1 + docs/reference/search/profile.asciidoc | 1 + docs/reference/search/request-body.asciidoc | 2 + .../suggesters/completion-suggest.asciidoc | 2 + docs/reference/search/uri-request.asciidoc | 1 + .../search/140_pre_filter_search_shards.yml | 39 ++++++++++++++++++- 18 files changed, 91 insertions(+), 19 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index 71a23fd266253..a51d97c0eddc6 100644 --- a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -321,16 +321,9 @@ private void executeSearch(SearchTask task, SearchTimeProvider timeProvider, Sea private boolean shouldPreFilterSearchShards(SearchRequest searchRequest, GroupShardsIterator shardIterators) { SearchSourceBuilder source = searchRequest.source(); - if (source == null) { - return false; - } else if (source.aggregations() != null && source.aggregations().hasGlobalAggregationBuilder()) { - return false; - } else if (source.query() == null) { - // match all in this case - return false; - } return searchRequest.searchType() == QUERY_THEN_FETCH && // we can't do this for DFS it needs to fan out // to all shards all the time + SearchService.canRewriteToMatchNone(source) && searchRequest.getPreFilterShardsAfter() < shardIterators.size(); } diff --git a/core/src/main/java/org/elasticsearch/rest/action/RestActions.java b/core/src/main/java/org/elasticsearch/rest/action/RestActions.java index 5a77a1042008b..61e3ded6456b6 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/RestActions.java +++ b/core/src/main/java/org/elasticsearch/rest/action/RestActions.java @@ -74,7 +74,7 @@ public static long parseVersion(RestRequest request, long defaultVersion) { public static void buildBroadcastShardsHeader(XContentBuilder builder, Params params, BroadcastResponse response) throws IOException { buildBroadcastShardsHeader(builder, params, - response.getTotalShards(), response.getSuccessfulShards(), 0, response.getFailedShards(), + response.getTotalShards(), response.getSuccessfulShards(), -1, response.getFailedShards(), response.getShardFailures()); } @@ -84,7 +84,9 @@ public static void buildBroadcastShardsHeader(XContentBuilder builder, Params pa builder.startObject(_SHARDS_FIELD.getPreferredName()); builder.field(TOTAL_FIELD.getPreferredName(), total); builder.field(SUCCESSFUL_FIELD.getPreferredName(), successful); - builder.field(SKIPPED_FIELD.getPreferredName(), skipped); + if (skipped >= 0) { + builder.field(SKIPPED_FIELD.getPreferredName(), skipped); + } builder.field(FAILED_FIELD.getPreferredName(), failed); if (shardFailures != null && shardFailures.length > 0) { builder.startArray(FAILURES_FIELD.getPreferredName()); diff --git a/core/src/main/java/org/elasticsearch/search/SearchService.java b/core/src/main/java/org/elasticsearch/search/SearchService.java index 2637081c43b77..221a27e11af56 100644 --- a/core/src/main/java/org/elasticsearch/search/SearchService.java +++ b/core/src/main/java/org/elasticsearch/search/SearchService.java @@ -42,6 +42,7 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.query.InnerHitContextBuilder; +import org.elasticsearch.index.query.MatchAllQueryBuilder; import org.elasticsearch.index.query.MatchNoneQueryBuilder; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryShardContext; @@ -843,15 +844,25 @@ public AliasFilter buildAliasFilter(ClusterState state, String index, String... public boolean canMatch(ShardSearchRequest request) throws IOException { try (DefaultSearchContext context = createSearchContext(request, defaultSearchTimeout, null)) { SearchSourceBuilder source = context.request().source(); - if (source != null) { + if (canRewriteToMatchNone(source)) { QueryBuilder queryBuilder = source.query(); - AggregatorFactories.Builder aggregations = source.aggregations(); - boolean hasGlobalAggs = aggregations != null && aggregations.hasGlobalAggregationBuilder(); - if (queryBuilder != null && hasGlobalAggs == false) { // we need to executed hasGlobalAggs is equivalent to match all - return queryBuilder instanceof MatchNoneQueryBuilder == false; - } + return queryBuilder instanceof MatchNoneQueryBuilder == false; } return true; // null query means match_all } } + + public static boolean canRewriteToMatchNone(SearchSourceBuilder source) { + if (source == null || source.query() == null || source.query() instanceof MatchAllQueryBuilder) { + return false; + } else { + AggregatorFactories.Builder aggregations = source.aggregations(); + if (aggregations != null) { + if (aggregations.mustVisiteAllDocs()) { + return false; + } + } + } + return true; + } } diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java b/core/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java index 95b5ebec6b0c6..f995c484d8273 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java @@ -27,6 +27,9 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.query.QueryRewriteContext; import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregationBuilder; +import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder; +import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregationBuilder; +import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.elasticsearch.search.aggregations.support.AggregationPath; import org.elasticsearch.search.aggregations.support.AggregationPath.PathElement; @@ -283,15 +286,22 @@ public void writeTo(StreamOutput out) throws IOException { } } - public boolean hasGlobalAggregationBuilder() { + public boolean mustVisiteAllDocs() { for (AggregationBuilder builder : aggregationBuilders) { if (builder instanceof GlobalAggregationBuilder) { return true; + } else if (builder instanceof TermsAggregationBuilder) { + if (((TermsAggregationBuilder) builder).minDocCount() == 0) { + return true; + } } + } return false; } + + public Builder addAggregator(AggregationBuilder factory) { if (!names.add(factory.name)) { throw new IllegalArgumentException("Two sibling aggregations cannot have the same name: [" + factory.name + "]"); diff --git a/docs/reference/aggregations/bucket/children-aggregation.asciidoc b/docs/reference/aggregations/bucket/children-aggregation.asciidoc index c5f706f0805c4..0503558ed236f 100644 --- a/docs/reference/aggregations/bucket/children-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/children-aggregation.asciidoc @@ -139,6 +139,7 @@ Possible response: "_shards": { "total": 5, "successful": 5, + "skipped" : 0, "failed": 0 }, "hits": { diff --git a/docs/reference/analysis/charfilters/pattern-replace-charfilter.asciidoc b/docs/reference/analysis/charfilters/pattern-replace-charfilter.asciidoc index 8221ae7cf3f4f..9386d7d9d354c 100644 --- a/docs/reference/analysis/charfilters/pattern-replace-charfilter.asciidoc +++ b/docs/reference/analysis/charfilters/pattern-replace-charfilter.asciidoc @@ -237,6 +237,7 @@ The output from the above is: "_shards": { "total": 5, "successful": 5, + "skipped" : 0, "failed": 0 }, "hits": { diff --git a/docs/reference/analysis/tokenizers/edgengram-tokenizer.asciidoc b/docs/reference/analysis/tokenizers/edgengram-tokenizer.asciidoc index b43b4518b8d72..5c2359163f927 100644 --- a/docs/reference/analysis/tokenizers/edgengram-tokenizer.asciidoc +++ b/docs/reference/analysis/tokenizers/edgengram-tokenizer.asciidoc @@ -296,6 +296,7 @@ GET my_index/_search "_shards": { "total": 5, "successful": 5, + "skipped" : 0, "failed": 0 }, "hits": { diff --git a/docs/reference/getting-started.asciidoc b/docs/reference/getting-started.asciidoc index d7db378afa42c..00fb08e3e634f 100755 --- a/docs/reference/getting-started.asciidoc +++ b/docs/reference/getting-started.asciidoc @@ -707,6 +707,7 @@ And the response (partially shown): "_shards" : { "total" : 5, "successful" : 5, + "skipped" : 0, "failed" : 0 }, "hits" : { @@ -774,6 +775,7 @@ to clutter the docs with it: "_shards" : { "total" : 5, "successful" : 5, + "skipped" : 0, "failed" : 0 }, "hits" : { @@ -1104,6 +1106,7 @@ And the response (partially shown): "_shards": { "total": 5, "successful": 5, + "skipped" : 0, "failed": 0 }, "hits" : { diff --git a/docs/reference/how-to/recipes.asciidoc b/docs/reference/how-to/recipes.asciidoc index 913fb80bea652..e5d14d19f2e6a 100644 --- a/docs/reference/how-to/recipes.asciidoc +++ b/docs/reference/how-to/recipes.asciidoc @@ -84,6 +84,7 @@ GET index/_search "_shards": { "total": 5, "successful": 5, + "skipped" : 0, "failed": 0 }, "hits": { @@ -141,6 +142,7 @@ GET index/_search "_shards": { "total": 5, "successful": 5, + "skipped" : 0, "failed": 0 }, "hits": { @@ -197,6 +199,7 @@ GET index/_search "_shards": { "total": 5, "successful": 5, + "skipped" : 0, "failed": 0 }, "hits": { diff --git a/docs/reference/mapping/params/normalizer.asciidoc b/docs/reference/mapping/params/normalizer.asciidoc index 5901e0d630b32..44a27dab7589d 100644 --- a/docs/reference/mapping/params/normalizer.asciidoc +++ b/docs/reference/mapping/params/normalizer.asciidoc @@ -75,6 +75,7 @@ both index and query time. "_shards": { "total": 5, "successful": 5, + "skipped" : 0, "failed": 0 }, "hits": { @@ -135,6 +136,7 @@ returns "_shards": { "total": 5, "successful": 5, + "skipped" : 0, "failed": 0 }, "hits": { diff --git a/docs/reference/mapping/types/range.asciidoc b/docs/reference/mapping/types/range.asciidoc index 4ac7ec03f61ea..3fdd7576a1762 100644 --- a/docs/reference/mapping/types/range.asciidoc +++ b/docs/reference/mapping/types/range.asciidoc @@ -77,6 +77,7 @@ The result produced by the above query. "_shards" : { "total": 2, "successful": 2, + "skipped" : 0, "failed": 0 }, "hits" : { diff --git a/docs/reference/query-dsl/percolate-query.asciidoc b/docs/reference/query-dsl/percolate-query.asciidoc index 84b3cd5c44789..0451f46587d20 100644 --- a/docs/reference/query-dsl/percolate-query.asciidoc +++ b/docs/reference/query-dsl/percolate-query.asciidoc @@ -85,6 +85,7 @@ The above request will yield the following response: "_shards": { "total": 5, "successful": 5, + "skipped" : 0, "failed": 0 }, "hits": { @@ -283,6 +284,7 @@ This will yield the following response. "_shards": { "total": 5, "successful": 5, + "skipped" : 0, "failed": 0 }, "hits": { diff --git a/docs/reference/search/count.asciidoc b/docs/reference/search/count.asciidoc index d4117e4e96eb7..dbab691867f54 100644 --- a/docs/reference/search/count.asciidoc +++ b/docs/reference/search/count.asciidoc @@ -39,6 +39,7 @@ tweets from the twitter index for a certain user. The result is: "_shards" : { "total" : 5, "successful" : 5, + "skipped" : 0, "failed" : 0 } } diff --git a/docs/reference/search/profile.asciidoc b/docs/reference/search/profile.asciidoc index 12848262e458a..5bef95cf931b5 100644 --- a/docs/reference/search/profile.asciidoc +++ b/docs/reference/search/profile.asciidoc @@ -41,6 +41,7 @@ This will yield the following result: "_shards": { "total": 1, "successful": 1, + "skipped" : 0, "failed": 0 }, "hits": { diff --git a/docs/reference/search/request-body.asciidoc b/docs/reference/search/request-body.asciidoc index 8e90711df952e..151160a1dbc31 100644 --- a/docs/reference/search/request-body.asciidoc +++ b/docs/reference/search/request-body.asciidoc @@ -27,6 +27,7 @@ And here is a sample response: "_shards":{ "total" : 1, "successful" : 1, + "skipped" : 0, "failed" : 0 }, "hits":{ @@ -142,6 +143,7 @@ be set to `true` in the response. "_shards": { "total": 1, "successful": 1, + "skipped" : 0, "failed": 0 }, "hits": { diff --git a/docs/reference/search/suggesters/completion-suggest.asciidoc b/docs/reference/search/suggesters/completion-suggest.asciidoc index 3f21a18de72b4..1a42dedf47c74 100644 --- a/docs/reference/search/suggesters/completion-suggest.asciidoc +++ b/docs/reference/search/suggesters/completion-suggest.asciidoc @@ -175,6 +175,7 @@ returns this response: "_shards" : { "total" : 5, "successful" : 5, + "skipped" : 0, "failed" : 0 }, "hits": ... @@ -243,6 +244,7 @@ Which should look like: "_shards" : { "total" : 5, "successful" : 5, + "skipped" : 0, "failed" : 0 }, "hits": { diff --git a/docs/reference/search/uri-request.asciidoc b/docs/reference/search/uri-request.asciidoc index 99c526abfafe9..86a57b515ed7a 100644 --- a/docs/reference/search/uri-request.asciidoc +++ b/docs/reference/search/uri-request.asciidoc @@ -23,6 +23,7 @@ And here is a sample response: "_shards":{ "total" : 1, "successful" : 1, + "skipped" : 0, "failed" : 0 }, "hits":{ diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search/140_pre_filter_search_shards.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search/140_pre_filter_search_shards.yml index 14ffec3c5135a..6ae10c35a86b3 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search/140_pre_filter_search_shards.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search/140_pre_filter_search_shards.yml @@ -58,8 +58,7 @@ setup: index: index_1 type: doc id: 1 - body: { "created_at": "2016-01-01" } - + body: { "created_at": "2016-01-01"} - do: index: index: index_2 @@ -122,6 +121,42 @@ setup: - match: { _shards.failed: 0 } - match: { hits.total: 0 } + - do: + search: + pre_filter_shards_after: 1 + body: {"size" : 0, "query" : { "range" : { "created_at" : { "gte" : "2016-02-01", "lt": "2018-02-01"} } }, "aggs" : { "some_agg" : { "global" : {} }}} + + - match: { _shards.total: 3 } + - match: { _shards.successful: 3 } + - match: { _shards.skipped : 0 } + - match: { _shards.failed: 0 } + - match: { hits.total: 2 } + - match: { aggregations.some_agg.doc_count: 3 } + + + - do: + search: + pre_filter_shards_after: 1 + body: { "size" : 0, "query" : { "range" : { "created_at" : { "gte" : "2016-02-01", "lt": "2018-02-01"}}}, "aggs" : { "idx_terms" : { "terms" : { "field" : "_index", "min_doc_count" : 0 } } } } + + - match: { _shards.total: 3 } + - match: { _shards.successful: 3 } + - match: { _shards.skipped : 0 } + - match: { _shards.failed: 0 } + - match: { hits.total: 2 } + - length: { aggregations.idx_terms.buckets: 3 } + + - do: + search: + pre_filter_shards_after: 1 + body: { "size" : 0, "query" : { "range" : { "created_at" : { "gte" : "2016-02-01", "lt": "2018-02-01"}}}, "aggs" : { "idx_terms" : { "terms" : { "field" : "_index" } } } } + + - match: { _shards.total: 3 } + - match: { _shards.successful: 3 } + - match: { _shards.skipped : 1 } + - match: { _shards.failed: 0 } + - match: { hits.total: 2 } + - length: { aggregations.idx_terms.buckets: 2 } From a97639c6ce729472a925b27e1131c4ed6114465d Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 12 Jul 2017 15:31:48 +0200 Subject: [PATCH 08/12] change parameter --- .../action/search/SearchRequest.java | 30 +++++++++---------- .../action/search/SearchRequestBuilder.java | 6 ++-- .../action/search/TransportSearchAction.java | 2 +- .../action/search/RestMultiSearchAction.java | 6 ++-- .../rest/action/search/RestSearchAction.java | 2 +- .../resources/rest-api-spec/api/msearch.json | 2 +- .../resources/rest-api-spec/api/search.json | 2 +- .../search/140_pre_filter_search_shards.yml | 20 ++++++------- .../test/client/RandomizingClient.java | 10 +++---- 9 files changed, 40 insertions(+), 40 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchRequest.java b/core/src/main/java/org/elasticsearch/action/search/SearchRequest.java index 3522d2d7b9776..19556d1e726f1 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchRequest.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchRequest.java @@ -58,7 +58,7 @@ public final class SearchRequest extends ActionRequest implements IndicesRequest private static final ToXContent.Params FORMAT_PARAMS = new ToXContent.MapParams(Collections.singletonMap("pretty", "false")); - public static final int DEFAULT_PRE_FILTER_SHARDS_AFTER = 128; + public static final int DEFAULT_PRE_FILTER_SHARD_SIZE = 128; private SearchType searchType = SearchType.DEFAULT; @@ -79,7 +79,7 @@ public final class SearchRequest extends ActionRequest implements IndicesRequest private int maxConcurrentShardRequests = 0; - private int preFilterShardsAfter = DEFAULT_PRE_FILTER_SHARDS_AFTER; + private int preFilterShardSize = DEFAULT_PRE_FILTER_SHARD_SIZE; private String[] types = Strings.EMPTY_ARRAY; @@ -330,26 +330,26 @@ public void setMaxConcurrentShardRequests(int maxConcurrentShardRequests) { this.maxConcurrentShardRequests = maxConcurrentShardRequests; } /** - * Sets a threshold that enforces a pre-filter roundtrip to prefilter search shards based on query rewriting if the number of shards + * Sets a threshold that enforces a pre-filter roundtrip to pre-filter search shards based on query rewriting if the number of shards * the search request expands to exceeds the threshold. This filter roundtrip can limit the number of shards significantly if for * instance a shard can not match any documents based on it's rewrite method ie. if date filters are mandatory to match but the shard * bounds and the query are disjoint. The default is 128 */ - public void setPreFilterSearchShardsAfter(int preFilterShardsAfter) { - if (preFilterShardsAfter < 1) { - throw new IllegalArgumentException("preFilterShardsAfter must be >= 1"); + public void setPreFilterShardSize(int preFilterShardSize) { + if (preFilterShardSize < 1) { + throw new IllegalArgumentException("preFilterShardSize must be >= 1"); } - this.preFilterShardsAfter = preFilterShardsAfter; + this.preFilterShardSize = preFilterShardSize; } /** - * Returns a threshold that enforces a pre-filter roundtrip to prefilter search shards based on query rewriting if the number of shards + * Returns a threshold that enforces a pre-filter roundtrip to pre-filter search shards based on query rewriting if the number of shards * the search request expands to exceeds the threshold. This filter roundtrip can limit the number of shards significantly if for * instance a shard can not match any documents based on it's rewrite method ie. if date filters are mandatory to match but the shard * bounds and the query are disjoint. The default is 128 */ - public int getPreFilterShardsAfter() { - return preFilterShardsAfter; + public int getPreFilterShardSize() { + return preFilterShardSize; } /** @@ -408,7 +408,7 @@ public void readFrom(StreamInput in) throws IOException { batchedReduceSize = in.readVInt(); if (in.getVersion().onOrAfter(Version.V_6_0_0_beta1)) { maxConcurrentShardRequests = in.readVInt(); - preFilterShardsAfter = in.readVInt(); + preFilterShardSize = in.readVInt(); } } @@ -430,7 +430,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeVInt(batchedReduceSize); if (out.getVersion().onOrAfter(Version.V_6_0_0_beta1)) { out.writeVInt(maxConcurrentShardRequests); - out.writeVInt(preFilterShardsAfter); + out.writeVInt(preFilterShardSize); } } @@ -453,14 +453,14 @@ public boolean equals(Object o) { Arrays.equals(types, that.types) && Objects.equals(batchedReduceSize, that.batchedReduceSize) && Objects.equals(maxConcurrentShardRequests, that.maxConcurrentShardRequests) && - Objects.equals(preFilterShardsAfter, that.preFilterShardsAfter) && + Objects.equals(preFilterShardSize, that.preFilterShardSize) && Objects.equals(indicesOptions, that.indicesOptions); } @Override public int hashCode() { return Objects.hash(searchType, Arrays.hashCode(indices), routing, preference, source, requestCache, - scroll, Arrays.hashCode(types), indicesOptions, batchedReduceSize, maxConcurrentShardRequests, preFilterShardsAfter); + scroll, Arrays.hashCode(types), indicesOptions, batchedReduceSize, maxConcurrentShardRequests, preFilterShardSize); } @Override @@ -476,7 +476,7 @@ public String toString() { ", scroll=" + scroll + ", maxConcurrentShardRequests=" + maxConcurrentShardRequests + ", batchedReduceSize=" + batchedReduceSize + - ", preFilterShardsAfter=" + preFilterShardsAfter + + ", preFilterShardSize=" + preFilterShardSize + ", source=" + source + '}'; } } diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchRequestBuilder.java b/core/src/main/java/org/elasticsearch/action/search/SearchRequestBuilder.java index f236c417c690d..92c1be01626a9 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchRequestBuilder.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchRequestBuilder.java @@ -537,13 +537,13 @@ public SearchRequestBuilder setMaxConcurrentShardRequests(int maxConcurrentShard } /** - * Sets a threshold that enforces a pre-filter roundtrip to prefilter search shards based on query rewriting if the number of shards + * Sets a threshold that enforces a pre-filter roundtrip to pre-filter search shards based on query rewriting if the number of shards * the search request expands to exceeds the threshold. This filter roundtrip can limit the number of shards significantly if for * instance a shard can not match any documents based on it's rewrite method ie. if date filters are mandatory to match but the shard * bounds and the query are disjoint. The default is 128 */ - public SearchRequestBuilder setPreFilterShardsAfter(int preFilterShardsAfter) { - this.request.setPreFilterSearchShardsAfter(preFilterShardsAfter); + public SearchRequestBuilder setPreFilterShardSize(int preFilterShardSize) { + this.request.setPreFilterShardSize(preFilterShardSize); return this; } } diff --git a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index a51d97c0eddc6..965d712e170a9 100644 --- a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -324,7 +324,7 @@ private boolean shouldPreFilterSearchShards(SearchRequest searchRequest, GroupSh return searchRequest.searchType() == QUERY_THEN_FETCH && // we can't do this for DFS it needs to fan out // to all shards all the time SearchService.canRewriteToMatchNone(source) && - searchRequest.getPreFilterShardsAfter() < shardIterators.size(); + searchRequest.getPreFilterShardSize() < shardIterators.size(); } static GroupShardsIterator mergeShardsIterators(GroupShardsIterator localShardsIterator, diff --git a/core/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java b/core/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java index 267278de4f930..c8550d40875a5 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java @@ -89,7 +89,7 @@ public static MultiSearchRequest parseRequest(RestRequest restRequest, boolean a multiRequest.maxConcurrentSearchRequests(restRequest.paramAsInt("max_concurrent_searches", 0)); } - int preFilterShardsAfter = restRequest.paramAsInt("pre_filter_shards_after", SearchRequest.DEFAULT_PRE_FILTER_SHARDS_AFTER); + int preFilterShardSize = restRequest.paramAsInt("pre_filter_shard_size", SearchRequest.DEFAULT_PRE_FILTER_SHARD_SIZE); parseMultiLineRequest(restRequest, multiRequest.indicesOptions(), allowExplicitIndex, (searchRequest, parser) -> { @@ -101,10 +101,10 @@ public static MultiSearchRequest parseRequest(RestRequest restRequest, boolean a } }); List requests = multiRequest.requests(); - preFilterShardsAfter = Math.max(1, preFilterShardsAfter / (requests.size()+1)); + preFilterShardSize = Math.max(1, preFilterShardSize / (requests.size()+1)); for (SearchRequest request : requests) { // preserve if it's set on the request - request.setPreFilterSearchShardsAfter(Math.min(preFilterShardsAfter, request.getPreFilterShardsAfter())); + request.setPreFilterShardSize(Math.min(preFilterShardSize, request.getPreFilterShardSize())); } return multiRequest; } diff --git a/core/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java b/core/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java index 0973db28f8f43..2e97560cf789f 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java @@ -98,7 +98,7 @@ public static void parseSearchRequest(SearchRequest searchRequest, RestRequest r final int batchedReduceSize = request.paramAsInt("batched_reduce_size", searchRequest.getBatchedReduceSize()); searchRequest.setBatchedReduceSize(batchedReduceSize); - searchRequest.setPreFilterSearchShardsAfter(request.paramAsInt("pre_filter_shards_after", searchRequest.getPreFilterShardsAfter())); + searchRequest.setPreFilterShardSize(request.paramAsInt("pre_filter_shard_size", searchRequest.getPreFilterShardSize())); if (request.hasParam("max_concurrent_shard_requests")) { // only set if we have the parameter since we auto adjust the max concurrency on the coordinator diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/msearch.json b/rest-api-spec/src/main/resources/rest-api-spec/api/msearch.json index 8b3618953a5ba..090c429fd82c0 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/msearch.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/msearch.json @@ -29,7 +29,7 @@ "type" : "boolean", "description" : "Specify whether aggregation and suggester names should be prefixed by their respective types in the response" }, - "pre_filter_shards_after" : { + "pre_filter_shard_size" : { "type" : "number", "description" : "A threshold that enforces a pre-filter roundtrip to prefilter search shards based on query rewriting if the number of shards the search request expands to exceeds the threshold. This filter roundtrip can limit the number of shards significantly if for instance a shard can not match any documents based on it's rewrite method ie. if date filters are mandatory to match but the shard bounds and the query are disjoint.", "default" : 128 diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/search.json b/rest-api-spec/src/main/resources/rest-api-spec/api/search.json index e2df23605b4aa..0f2beffa4579b 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/search.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/search.json @@ -169,7 +169,7 @@ "description" : "The number of concurrent shard requests this search executes concurrently. This value should be used to limit the impact of the search on the cluster in order to limit the number of concurrent shard requests", "default" : "The default grows with the number of nodes in the cluster but is at most 256." }, - "pre_filter_shards_after" : { + "pre_filter_shard_size" : { "type" : "number", "description" : "A threshold that enforces a pre-filter roundtrip to prefilter search shards based on query rewriting if the number of shards the search request expands to exceeds the threshold. This filter roundtrip can limit the number of shards significantly if for instance a shard can not match any documents based on it's rewrite method ie. if date filters are mandatory to match but the shard bounds and the query are disjoint.", "default" : 128 diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search/140_pre_filter_search_shards.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search/140_pre_filter_search_shards.yml index 6ae10c35a86b3..2e8553e1425c6 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search/140_pre_filter_search_shards.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search/140_pre_filter_search_shards.yml @@ -38,18 +38,18 @@ setup: --- -"pre_filter_shards_after with invalid parameter": +"pre_filter_shard_size with invalid parameter": - skip: version: " - 5.99.99" reason: this was added in 6.0.0 - do: - catch: /preFilterShardsAfter must be >= 1/ + catch: /preFilterShardSizemust be >= 1/ search: index: test_1 - pre_filter_shards_after: 0 + pre_filter_shard_size: 0 --- -"pre_filter_shards_after with shards that have no hit": +"pre_filter_shard_size with shards that have no hit": - skip: version: " - 5.99.99" reason: this was added in 6.0.0 @@ -89,7 +89,7 @@ setup: # this is the case where we have an empty body and don't skip anything since it's match_all - do: search: - pre_filter_shards_after: 1 + pre_filter_shard_size: 1 - match: { _shards.total: 3 } - match: { _shards.successful: 3 } @@ -100,7 +100,7 @@ setup: # this is a case where we can actually skip due to rewrite - do: search: - pre_filter_shards_after: 1 + pre_filter_shard_size: 1 body: { "size" : 0, "query" : { "range" : { "created_at" : { "gte" : "2016-02-01", "lt": "2018-02-01"} } } } - match: { _shards.total: 3 } @@ -112,7 +112,7 @@ setup: # this case we skip all except of one since we need a real result - do: search: - pre_filter_shards_after: 1 + pre_filter_shard_size: 1 body: { "size" : 0, "query" : { "range" : { "created_at" : { "gte" : "2019-02-01", "lt": "2020-02-01"} } } } - match: { _shards.total: 3 } @@ -123,7 +123,7 @@ setup: - do: search: - pre_filter_shards_after: 1 + pre_filter_shard_size: 1 body: {"size" : 0, "query" : { "range" : { "created_at" : { "gte" : "2016-02-01", "lt": "2018-02-01"} } }, "aggs" : { "some_agg" : { "global" : {} }}} - match: { _shards.total: 3 } @@ -136,7 +136,7 @@ setup: - do: search: - pre_filter_shards_after: 1 + pre_filter_shard_size: 1 body: { "size" : 0, "query" : { "range" : { "created_at" : { "gte" : "2016-02-01", "lt": "2018-02-01"}}}, "aggs" : { "idx_terms" : { "terms" : { "field" : "_index", "min_doc_count" : 0 } } } } - match: { _shards.total: 3 } @@ -148,7 +148,7 @@ setup: - do: search: - pre_filter_shards_after: 1 + pre_filter_shard_size: 1 body: { "size" : 0, "query" : { "range" : { "created_at" : { "gte" : "2016-02-01", "lt": "2018-02-01"}}}, "aggs" : { "idx_terms" : { "terms" : { "field" : "_index" } } } } - match: { _shards.total: 3 } diff --git a/test/framework/src/main/java/org/elasticsearch/test/client/RandomizingClient.java b/test/framework/src/main/java/org/elasticsearch/test/client/RandomizingClient.java index 3bf89247a539d..6759461868a4a 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/client/RandomizingClient.java +++ b/test/framework/src/main/java/org/elasticsearch/test/client/RandomizingClient.java @@ -38,7 +38,7 @@ public class RandomizingClient extends FilterClient { private final String defaultPreference; private final int batchedReduceSize; private final int maxConcurrentShardRequests; - private final int preFilterAfter; + private final int preFilterShardSize; public RandomizingClient(Client client, Random random) { @@ -63,9 +63,9 @@ public RandomizingClient(Client client, Random random) { this.maxConcurrentShardRequests = -1; // randomly use the default } if (random.nextBoolean()) { - preFilterAfter = 1 + random.nextInt(1 << random.nextInt(7)); + preFilterShardSize = 1 + random.nextInt(1 << random.nextInt(7)); } else { - preFilterAfter = -1; + preFilterShardSize = -1; } } @@ -76,8 +76,8 @@ public SearchRequestBuilder prepareSearch(String... indices) { if (maxConcurrentShardRequests != -1) { searchRequestBuilder.setMaxConcurrentShardRequests(maxConcurrentShardRequests); } - if (preFilterAfter != -1) { - searchRequestBuilder.setPreFilterShardsAfter(preFilterAfter); + if (preFilterShardSize != -1) { + searchRequestBuilder.setPreFilterShardSize(preFilterShardSize); } return searchRequestBuilder; } From 807e7e2e7eea0a1f178aa678649a285a6f3a5345 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 12 Jul 2017 15:57:12 +0200 Subject: [PATCH 09/12] fix compilation and add test for SearchService --- .../action/search/TransportSearchAction.java | 3 +- .../elasticsearch/search/SearchService.java | 2 + .../indices/IndicesRequestCacheIT.java | 6 +-- .../search/SearchServiceTests.java | 50 +++++++++++++++++++ 4 files changed, 56 insertions(+), 5 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index 965d712e170a9..21a1097f8fbbd 100644 --- a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -321,8 +321,7 @@ private void executeSearch(SearchTask task, SearchTimeProvider timeProvider, Sea private boolean shouldPreFilterSearchShards(SearchRequest searchRequest, GroupShardsIterator shardIterators) { SearchSourceBuilder source = searchRequest.source(); - return searchRequest.searchType() == QUERY_THEN_FETCH && // we can't do this for DFS it needs to fan out - // to all shards all the time + return searchRequest.searchType() == QUERY_THEN_FETCH && // we can't do this for DFS it needs to fan out to all shards all the time SearchService.canRewriteToMatchNone(source) && searchRequest.getPreFilterShardSize() < shardIterators.size(); } diff --git a/core/src/main/java/org/elasticsearch/search/SearchService.java b/core/src/main/java/org/elasticsearch/search/SearchService.java index 221a27e11af56..707312cd2814c 100644 --- a/core/src/main/java/org/elasticsearch/search/SearchService.java +++ b/core/src/main/java/org/elasticsearch/search/SearchService.java @@ -26,6 +26,7 @@ import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.search.SearchTask; +import org.elasticsearch.action.search.SearchType; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Nullable; @@ -842,6 +843,7 @@ public AliasFilter buildAliasFilter(ClusterState state, String index, String... * shard. */ public boolean canMatch(ShardSearchRequest request) throws IOException { + assert request.searchType() == SearchType.QUERY_THEN_FETCH : "unexpected search type: " + request.searchType(); try (DefaultSearchContext context = createSearchContext(request, defaultSearchTimeout, null)) { SearchSourceBuilder source = context.request().source(); if (canRewriteToMatchNone(source)) { diff --git a/core/src/test/java/org/elasticsearch/indices/IndicesRequestCacheIT.java b/core/src/test/java/org/elasticsearch/indices/IndicesRequestCacheIT.java index e8060b4abf2c9..534bd23ea5de9 100644 --- a/core/src/test/java/org/elasticsearch/indices/IndicesRequestCacheIT.java +++ b/core/src/test/java/org/elasticsearch/indices/IndicesRequestCacheIT.java @@ -111,7 +111,7 @@ public void testQueryRewrite() throws Exception { equalTo(0L)); final SearchResponse r1 = client().prepareSearch("index").setSearchType(SearchType.QUERY_THEN_FETCH).setSize(0) - .setQuery(QueryBuilders.rangeQuery("s").gte("2016-03-19").lte("2016-03-25")).setPreFilterShardsAfter(Integer.MAX_VALUE) + .setQuery(QueryBuilders.rangeQuery("s").gte("2016-03-19").lte("2016-03-25")).setPreFilterShardSize(Integer.MAX_VALUE) .get(); assertSearchResponse(r1); assertThat(r1.getHits().getTotalHits(), equalTo(7L)); @@ -122,7 +122,7 @@ public void testQueryRewrite() throws Exception { final SearchResponse r2 = client().prepareSearch("index").setSearchType(SearchType.QUERY_THEN_FETCH).setSize(0) .setQuery(QueryBuilders.rangeQuery("s").gte("2016-03-20").lte("2016-03-26")) - .setPreFilterShardsAfter(Integer.MAX_VALUE).get(); + .setPreFilterShardSize(Integer.MAX_VALUE).get(); assertSearchResponse(r2); assertThat(r2.getHits().getTotalHits(), equalTo(7L)); assertThat(client().admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), @@ -131,7 +131,7 @@ public void testQueryRewrite() throws Exception { equalTo(7L)); final SearchResponse r3 = client().prepareSearch("index").setSearchType(SearchType.QUERY_THEN_FETCH).setSize(0) - .setQuery(QueryBuilders.rangeQuery("s").gte("2016-03-21").lte("2016-03-27")).setPreFilterShardsAfter(Integer.MAX_VALUE) + .setQuery(QueryBuilders.rangeQuery("s").gte("2016-03-21").lte("2016-03-27")).setPreFilterShardSize(Integer.MAX_VALUE) .get(); assertSearchResponse(r3); assertThat(r3.getHits().getTotalHits(), equalTo(7L)); diff --git a/core/src/test/java/org/elasticsearch/search/SearchServiceTests.java b/core/src/test/java/org/elasticsearch/search/SearchServiceTests.java index 864a1806b6e99..9d0e978795c70 100644 --- a/core/src/test/java/org/elasticsearch/search/SearchServiceTests.java +++ b/core/src/test/java/org/elasticsearch/search/SearchServiceTests.java @@ -25,6 +25,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchPhaseExecutionException; +import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchTask; import org.elasticsearch.action.search.SearchType; @@ -37,13 +38,19 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.query.AbstractQueryBuilder; +import org.elasticsearch.index.query.MatchAllQueryBuilder; +import org.elasticsearch.index.query.MatchNoneQueryBuilder; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryRewriteContext; import org.elasticsearch.index.query.QueryShardContext; +import org.elasticsearch.index.query.TermQueryBuilder; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.SearchPlugin; +import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregationBuilder; +import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder; +import org.elasticsearch.search.aggregations.support.ValueType; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.fetch.ShardFetchRequest; import org.elasticsearch.search.internal.AliasFilter; @@ -305,4 +312,47 @@ public String getWriteableName() { return null; } } + + public void testCanMatch() throws IOException { + createIndex("index"); + final SearchService service = getInstanceFromNode(SearchService.class); + final IndicesService indicesService = getInstanceFromNode(IndicesService.class); + final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); + final IndexShard indexShard = indexService.getShard(0); + assertTrue(service.canMatch(new ShardSearchLocalRequest(indexShard.shardId(), 1, SearchType.QUERY_THEN_FETCH, null, + Strings.EMPTY_ARRAY, false, new AliasFilter(null, Strings.EMPTY_ARRAY), 1f))); + + assertTrue(service.canMatch(new ShardSearchLocalRequest(indexShard.shardId(), 1, SearchType.QUERY_THEN_FETCH, + new SearchSourceBuilder(), Strings.EMPTY_ARRAY, false, new AliasFilter(null, Strings.EMPTY_ARRAY), 1f))); + + assertTrue(service.canMatch(new ShardSearchLocalRequest(indexShard.shardId(), 1, SearchType.QUERY_THEN_FETCH, + new SearchSourceBuilder().query(new MatchAllQueryBuilder()), Strings.EMPTY_ARRAY, false, + new AliasFilter(null, Strings.EMPTY_ARRAY), 1f))); + + assertTrue(service.canMatch(new ShardSearchLocalRequest(indexShard.shardId(), 1, SearchType.QUERY_THEN_FETCH, + new SearchSourceBuilder().query(new MatchNoneQueryBuilder()) + .aggregation(new TermsAggregationBuilder("test", ValueType.STRING).minDocCount(0)), Strings.EMPTY_ARRAY, false, + new AliasFilter(null, Strings.EMPTY_ARRAY), 1f))); + assertTrue(service.canMatch(new ShardSearchLocalRequest(indexShard.shardId(), 1, SearchType.QUERY_THEN_FETCH, + new SearchSourceBuilder().query(new MatchNoneQueryBuilder()) + .aggregation(new GlobalAggregationBuilder("test")), Strings.EMPTY_ARRAY, false, + new AliasFilter(null, Strings.EMPTY_ARRAY), 1f))); + + assertFalse(service.canMatch(new ShardSearchLocalRequest(indexShard.shardId(), 1, SearchType.QUERY_THEN_FETCH, + new SearchSourceBuilder().query(new MatchNoneQueryBuilder()), Strings.EMPTY_ARRAY, false, + new AliasFilter(null, Strings.EMPTY_ARRAY), 1f))); + + } + + public void testCanRewriteToMatchNone() { + assertFalse(SearchService.canRewriteToMatchNone(new SearchSourceBuilder().query(new MatchNoneQueryBuilder()) + .aggregation(new GlobalAggregationBuilder("test")))); + assertFalse(SearchService.canRewriteToMatchNone(new SearchSourceBuilder())); + assertFalse(SearchService.canRewriteToMatchNone(null)); + assertFalse(SearchService.canRewriteToMatchNone(new SearchSourceBuilder().query(new MatchNoneQueryBuilder()) + .aggregation(new TermsAggregationBuilder("test", ValueType.STRING).minDocCount(0)))); + assertTrue(SearchService.canRewriteToMatchNone(new SearchSourceBuilder().query(new TermQueryBuilder("foo", "bar")))); + assertTrue(SearchService.canRewriteToMatchNone(new SearchSourceBuilder().query(new MatchNoneQueryBuilder()) + .aggregation(new TermsAggregationBuilder("test", ValueType.STRING).minDocCount(1)))); + } } From ae0b64aaaa88dae9e10f97d79914cd3d25b1a391 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 12 Jul 2017 16:29:58 +0200 Subject: [PATCH 10/12] add unittest for filter search phase --- .../CanMatchPreFilterSearchPhaseTests.java | 156 ++++++++++++++++++ .../action/search/SearchAsyncActionTests.java | 2 +- .../search/140_pre_filter_search_shards.yml | 2 +- 3 files changed, 158 insertions(+), 2 deletions(-) create mode 100644 core/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java diff --git a/core/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java b/core/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java new file mode 100644 index 0000000000000..7c393b41c07d7 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java @@ -0,0 +1,156 @@ +/* + * 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.action.ActionListener; +import org.elasticsearch.action.OriginalIndices; +import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.routing.GroupShardsIterator; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.EsExecutors; +import org.elasticsearch.search.internal.AliasFilter; +import org.elasticsearch.search.internal.ShardSearchTransportRequest; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.transport.Transport; + +import java.io.IOException; +import java.util.Collections; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicReference; + +public class CanMatchPreFilterSearchPhaseTests extends ESTestCase { + + + public void testFilterShards() throws InterruptedException { + + final TransportSearchAction.SearchTimeProvider timeProvider = new TransportSearchAction.SearchTimeProvider(0, System.nanoTime(), + System::nanoTime); + + Map lookup = new ConcurrentHashMap<>(); + DiscoveryNode primaryNode = new DiscoveryNode("node_1", buildNewFakeTransportAddress(), Version.CURRENT); + DiscoveryNode replicaNode = new DiscoveryNode("node_2", buildNewFakeTransportAddress(), Version.CURRENT); + lookup.put("node1", new SearchAsyncActionTests.MockConnection(primaryNode)); + lookup.put("node2", new SearchAsyncActionTests.MockConnection(replicaNode)); + final boolean shard1 = randomBoolean(); + final boolean shard2 = randomBoolean(); + + SearchTransportService searchTransportService = new SearchTransportService( + Settings.builder().put("search.remote.connect", false).build(), null) { + + @Override + public void sendCanMatch(Transport.Connection connection, ShardSearchTransportRequest request, SearchTask task, + ActionListener listener) { + new Thread(() -> listener.onResponse(new CanMatchResponse(request.shardId().id() == 0 ? shard1 : + shard2))).start(); + } + }; + + AtomicReference> result = new AtomicReference<>(); + CountDownLatch latch = new CountDownLatch(1); + GroupShardsIterator shardsIter = SearchAsyncActionTests.getShardsIter("idx", + new OriginalIndices(new String[]{"idx"}, IndicesOptions.strictExpandOpenAndForbidClosed()), + 2, randomBoolean(), primaryNode, replicaNode); + CanMatchPreFilterSearchPhase canMatchPhase = new CanMatchPreFilterSearchPhase(logger, + searchTransportService, + (clusterAlias, node) -> lookup.get(node), + Collections.singletonMap("_na_", new AliasFilter(null, Strings.EMPTY_ARRAY)), + Collections.emptyMap(), EsExecutors.newDirectExecutorService(), + new SearchRequest(), null, shardsIter, timeProvider, 0, null, + (iter) -> new SearchPhase("test") { + @Override + public void run() throws IOException { + result.set(iter); + latch.countDown(); + }}); + + canMatchPhase.start(); + latch.await(); + + if (shard1 && shard2) { + for (SearchShardIterator i : result.get()) { + assertFalse(i.skip()); + } + } else if (shard1 == false && shard2 == false) { + assertFalse(result.get().get(0).skip()); + assertTrue(result.get().get(1).skip()); + } else { + assertEquals(0, result.get().get(0).shardId().id()); + assertEquals(1, result.get().get(1).shardId().id()); + assertEquals(shard1, !result.get().get(0).skip()); + assertEquals(shard2, !result.get().get(1).skip()); + } + } + + public void testFilterWithFailure() throws InterruptedException { + final TransportSearchAction.SearchTimeProvider timeProvider = new TransportSearchAction.SearchTimeProvider(0, System.nanoTime(), + System::nanoTime); + Map lookup = new ConcurrentHashMap<>(); + DiscoveryNode primaryNode = new DiscoveryNode("node_1", buildNewFakeTransportAddress(), Version.CURRENT); + DiscoveryNode replicaNode = new DiscoveryNode("node_2", buildNewFakeTransportAddress(), Version.CURRENT); + lookup.put("node1", new SearchAsyncActionTests.MockConnection(primaryNode)); + lookup.put("node2", new SearchAsyncActionTests.MockConnection(replicaNode)); + final boolean shard1 = randomBoolean(); + SearchTransportService searchTransportService = new SearchTransportService( + Settings.builder().put("search.remote.connect", false).build(), null) { + + @Override + public void sendCanMatch(Transport.Connection connection, ShardSearchTransportRequest request, SearchTask task, + ActionListener listener) { + new Thread(() -> { + if (request.shardId().id() == 0) { + listener.onResponse(new CanMatchResponse(shard1)); + } else { + listener.onFailure(new NullPointerException()); + } + }).start(); + } + }; + + AtomicReference> result = new AtomicReference<>(); + CountDownLatch latch = new CountDownLatch(1); + GroupShardsIterator shardsIter = SearchAsyncActionTests.getShardsIter("idx", + new OriginalIndices(new String[]{"idx"}, IndicesOptions.strictExpandOpenAndForbidClosed()), + 2, randomBoolean(), primaryNode, replicaNode); + CanMatchPreFilterSearchPhase canMatchPhase = new CanMatchPreFilterSearchPhase(logger, + searchTransportService, + (clusterAlias, node) -> lookup.get(node), + Collections.singletonMap("_na_", new AliasFilter(null, Strings.EMPTY_ARRAY)), + Collections.emptyMap(), EsExecutors.newDirectExecutorService(), + new SearchRequest(), null, shardsIter, timeProvider, 0, null, + (iter) -> new SearchPhase("test") { + @Override + public void run() throws IOException { + result.set(iter); + latch.countDown(); + }}); + + canMatchPhase.start(); + latch.await(); + + assertEquals(0, result.get().get(0).shardId().id()); + assertEquals(1, result.get().get(1).shardId().id()); + assertEquals(shard1, !result.get().get(0).skip()); + assertFalse(result.get().get(1).skip()); // never skip the failure + } +} diff --git a/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java b/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java index f4e45c300e728..be95fdcbf128b 100644 --- a/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java @@ -255,7 +255,7 @@ public void run() throws IOException { } } - private static GroupShardsIterator getShardsIter(String index, OriginalIndices originalIndices, int numShards, + static GroupShardsIterator getShardsIter(String index, OriginalIndices originalIndices, int numShards, boolean doReplicas, DiscoveryNode primaryNode, DiscoveryNode replicaNode) { ArrayList list = new ArrayList<>(); for (int i = 0; i < numShards; i++) { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search/140_pre_filter_search_shards.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search/140_pre_filter_search_shards.yml index 2e8553e1425c6..0095d2b1e9029 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search/140_pre_filter_search_shards.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search/140_pre_filter_search_shards.yml @@ -43,7 +43,7 @@ setup: version: " - 5.99.99" reason: this was added in 6.0.0 - do: - catch: /preFilterShardSizemust be >= 1/ + catch: /preFilterShardSize must be >= 1/ search: index: test_1 pre_filter_shard_size: 0 From 2dcd3e0a08ba625747db53082a0293c9b8dbacdc Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 12 Jul 2017 16:40:14 +0200 Subject: [PATCH 11/12] add unittest for skipping shards in the intitial search phase --- .../action/search/SearchShardIterator.java | 7 ++ .../action/search/SearchAsyncActionTests.java | 93 +++++++++++++++++++ 2 files changed, 100 insertions(+) diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchShardIterator.java b/core/src/main/java/org/elasticsearch/action/search/SearchShardIterator.java index 523faa3cb8d06..c36d2b7908f78 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchShardIterator.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchShardIterator.java @@ -60,11 +60,18 @@ public String getClusterAlias() { return clusterAlias; } + /** + * Reset the iterator and mark it as skippable + * @see #skip() + */ void resetAndSkip() { reset(); skip = true; } + /** + * Returns true if the search execution should skip this shard since it can not match any documents given the query. + */ boolean skip() { return skip; } diff --git a/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java b/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java index be95fdcbf128b..a2b438a66e418 100644 --- a/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java @@ -55,6 +55,99 @@ public class SearchAsyncActionTests extends ESTestCase { + public void testSkipSearchShards() throws InterruptedException { + SearchRequest request = new SearchRequest(); + CountDownLatch latch = new CountDownLatch(1); + AtomicReference response = new AtomicReference<>(); + ActionListener responseListener = new ActionListener() { + @Override + public void onResponse(SearchResponse searchResponse) { + response.set((TestSearchResponse) searchResponse); + } + + @Override + public void onFailure(Exception e) { + logger.warn("test failed", e); + fail(e.getMessage()); + } + }; + DiscoveryNode primaryNode = new DiscoveryNode("node_1", buildNewFakeTransportAddress(), Version.CURRENT); + DiscoveryNode replicaNode = new DiscoveryNode("node_2", buildNewFakeTransportAddress(), Version.CURRENT); + + AtomicInteger contextIdGenerator = new AtomicInteger(0); + GroupShardsIterator shardsIter = getShardsIter("idx", + new OriginalIndices(new String[]{"idx"}, IndicesOptions.strictExpandOpenAndForbidClosed()), + 10, randomBoolean(), primaryNode, replicaNode); + int numSkipped = 0; + for (SearchShardIterator iter : shardsIter) { + if (iter.shardId().id() % 2 == 0) { + iter.resetAndSkip(); + numSkipped++; + } + } + + SearchTransportService transportService = new SearchTransportService(Settings.EMPTY, null); + Map lookup = new HashMap<>(); + Map seenShard = new ConcurrentHashMap<>(); + lookup.put(primaryNode.getId(), new MockConnection(primaryNode)); + lookup.put(replicaNode.getId(), new MockConnection(replicaNode)); + Map aliasFilters = Collections.singletonMap("_na_", new AliasFilter(null, Strings.EMPTY_ARRAY)); + AtomicInteger numRequests = new AtomicInteger(0); + AbstractSearchAsyncAction asyncAction = + new AbstractSearchAsyncAction( + "test", + logger, + transportService, + (cluster, node) -> { + assert cluster == null : "cluster was not null: " + cluster; + return lookup.get(node); }, + aliasFilters, + Collections.emptyMap(), + null, + request, + responseListener, + shardsIter, + new TransportSearchAction.SearchTimeProvider(0, 0, () -> 0), + 0, + null, + new InitialSearchPhase.ArraySearchPhaseResults<>(shardsIter.size())) { + + @Override + protected void executePhaseOnShard(SearchShardIterator shardIt, ShardRouting shard, + SearchActionListener listener) { + seenShard.computeIfAbsent(shard.shardId(), (i) -> { + numRequests.incrementAndGet(); // only count this once per replica + return Boolean.TRUE; + }); + + new Thread(() -> { + Transport.Connection connection = getConnection(null, shard.currentNodeId()); + TestSearchPhaseResult testSearchPhaseResult = new TestSearchPhaseResult(contextIdGenerator.incrementAndGet(), + connection.getNode()); + listener.onResponse(testSearchPhaseResult); + + }).start(); + } + + @Override + protected SearchPhase getNextPhase(SearchPhaseResults results, SearchPhaseContext context) { + return new SearchPhase("test") { + @Override + public void run() throws IOException { + latch.countDown(); + } + }; + } + }; + asyncAction.start(); + latch.await(); + SearchResponse searchResponse = asyncAction.buildSearchResponse(null, null); + assertEquals(shardsIter.size()-numSkipped, numRequests.get()); + assertEquals(0, searchResponse.getFailedShards()); + assertEquals(numSkipped, searchResponse.getSkippedShards()); + assertEquals(shardsIter.size(), searchResponse.getSuccessfulShards()); + } + public void testLimitConcurrentShardRequests() throws InterruptedException { SearchRequest request = new SearchRequest(); int numConcurrent = randomIntBetween(1, 5); From 94d8713df0c168f88cd50de95188fcd091731813 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 12 Jul 2017 21:17:13 +0200 Subject: [PATCH 12/12] add test for cross cluster search to ensure shard skipping works --- .../test/multi_cluster/70_skip_shards.yml | 55 +++++++++++++++++++ .../test/remote_cluster/10_basic.yml | 8 ++- 2 files changed, 62 insertions(+), 1 deletion(-) create mode 100644 qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/70_skip_shards.yml diff --git a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/70_skip_shards.yml b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/70_skip_shards.yml new file mode 100644 index 0000000000000..e50b32c5e4b3c --- /dev/null +++ b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/70_skip_shards.yml @@ -0,0 +1,55 @@ +--- +"Test that remote indices are subject to shard skipping": + + - do: + indices.create: + index: skip_shards_index + body: + settings: + index: + number_of_shards: 1 + number_of_replicas: 0 + mappings: + test_type: + properties: + created_at: + type: date + format: "yyyy-MM-dd" + + - do: + bulk: + refresh: true + body: + - '{"index": {"_index": "skip_shards_index", "_type": "test_type"}}' + - '{"f1": "local_cluster", "sort_field": 0, "created_at" : "2017-01-01"}' + + # check that we skip the remote shard + - do: + search: + index: "skip_shards_index,my_remote_cluster:single_doc_index" + pre_filter_shard_size: 1 + body: { "size" : 10, "query" : { "range" : { "created_at" : { "gte" : "2016-02-01", "lt": "2018-02-01"} } } } + + - match: { hits.total: 1 } + - match: { hits.hits.0._index: "skip_shards_index"} + - match: { _shards.total: 2 } + - match: { _shards.successful: 2 } + - match: { _shards.skipped : 1} + - match: { _shards.failed: 0 } + - match: { hits.total: 1 } + + # check that we skip the local shard + - do: + search: + index: "skip_shards_index,my_remote_cluster:single_doc_index" + pre_filter_shard_size: 1 + body: { "size" : 10, "query" : { "range" : { "created_at" : { "gte" : "2015-02-01", "lt": "2016-02-01"} } } } + + - match: { hits.total: 1 } + - match: { hits.hits.0._index: "my_remote_cluster:single_doc_index"} + - match: { _shards.total: 2 } + - match: { _shards.successful: 2 } + - match: { _shards.skipped : 1} + - match: { _shards.failed: 0 } + - match: { hits.total: 1 } + diff --git a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/remote_cluster/10_basic.yml b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/remote_cluster/10_basic.yml index 340763aa5fe37..d37bb5a182586 100644 --- a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/remote_cluster/10_basic.yml +++ b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/remote_cluster/10_basic.yml @@ -9,13 +9,19 @@ index: number_of_shards: 1 number_of_replicas: 0 + mappings: + test_type: + properties: + created_at: + type: date + format: "yyyy-MM-dd" - do: bulk: refresh: true body: - '{"index": {"_index": "single_doc_index", "_type": "test_type"}}' - - '{"f1": "remote_cluster", "sort_field": 1}' + - '{"f1": "remote_cluster", "sort_field": 1, "created_at" : "2016-01-01"}' - do: indices.create: