From 28fa23b8cb69dc2e78d6aaf491d08d47b1164e43 Mon Sep 17 00:00:00 2001 From: jimczi Date: Thu, 30 Jan 2020 22:52:08 +0100 Subject: [PATCH 01/37] Always rewrite search shard request outside of the search thread pool This change ensures that the rewrite of the shard request is executed in the network thread or in the refresh listener when waiting for an active shard. This allows queries that rewrite to match_no_docs to bypass the search thread pool entirely even if the can_match phase was skipped (pre_filter_shard_size > number of shards). Coordinating nodes don't have the ability to create empty responses so this change also ensures that at least one shard creates a full empty response while the other can return null ones. This is needed since creating true empty responses on shards require to create concrete aggregators which would be too costly to build on a network thread. We should move this functionality to aggregation builders in a follow up but that would be a much bigger change. This change is also important for #49601 since we want to add the ability to use the result of other shards to rewrite the request of subsequent ones. For instance if the first M shards have their top N computed, the top worst document in the global queue can be pass to subsequent shards that can then rewrite to match_no_docs if they can guarantee that they don't have any document better than the provided one. --- .../search/140_pre_filter_search_shards.yml | 29 ++ .../search/AbstractSearchAsyncAction.java | 14 +- .../action/search/SearchPhaseController.java | 66 +++-- .../elasticsearch/index/shard/IndexShard.java | 7 + .../elasticsearch/search/SearchService.java | 199 ++++++++++--- .../search/internal/ShardSearchRequest.java | 26 +- .../search/query/QuerySearchResult.java | 55 +++- .../search/SearchPhaseControllerTests.java | 61 +++- .../indices/IndicesRequestCacheIT.java | 13 +- .../search/SearchServiceTests.java | 268 +++++++++++++----- .../index/engine/FrozenIndexTests.java | 38 ++- 11 files changed, 612 insertions(+), 164 deletions(-) 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 e9fb959406e0e..636f3d88e10cf 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 @@ -153,3 +153,32 @@ setup: - match: { _shards.failed: 0 } - match: { hits.total: 2 } - length: { aggregations.idx_terms.buckets: 2 } + + # check that empty responses are correctly handled when rewriting to match_no_docs + - do: + search: + rest_total_hits_as_int: true + # ensure that one shard can return empty response + max_concurrent_shard_requests: 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 : 0 } + - match: { _shards.failed: 0 } + - match: { hits.total: 2 } + - length: { aggregations.idx_terms.buckets: 2 } + + - do: + search: + rest_total_hits_as_int: true + # ensure that one shard can return empty response + max_concurrent_shard_requests: 2 + body: { "size" : 0, "query" : { "range" : { "created_at" : { "gte" : "2019-02-01"}}}, "aggs" : { "idx_terms" : { "terms" : { "field" : "_index" } } } } + + - match: { _shards.total: 3 } + - match: { _shards.successful: 3 } + - match: { _shards.skipped : 0 } + - match: { _shards.failed: 0 } + - match: { hits.total: 0 } + - length: { aggregations.idx_terms.buckets: 0 } diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index ca68bb4008146..bc7b347531ad0 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -51,6 +51,7 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executor; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiFunction; import java.util.stream.Collectors; @@ -75,13 +76,14 @@ abstract class AbstractSearchAsyncAction exten **/ private final BiFunction nodeIdToConnection; private final SearchTask task; - private final SearchPhaseResults results; + protected final SearchPhaseResults results; private final long clusterStateVersion; private final Map aliasFilter; private final Map concreteIndexBoosts; private final Map> indexRoutings; private final SetOnce> shardFailures = new SetOnce<>(); private final Object shardFailuresMutex = new Object(); + private final AtomicBoolean hasShardResponse = new AtomicBoolean(false); private final AtomicInteger successfulOps = new AtomicInteger(); private final AtomicInteger skippedOps = new AtomicInteger(); private final SearchTimeProvider timeProvider; @@ -462,9 +464,10 @@ public final void onShardFailure(final int shardIndex, @Nullable SearchShardTarg * @param result the result returned form the shard * @param shardIt the shard iterator */ - private void onShardResult(Result result, SearchShardIterator shardIt) { + protected void onShardResult(Result result, SearchShardIterator shardIt) { assert result.getShardIndex() != -1 : "shard index is not set"; assert result.getSearchShardTarget() != null : "search shard target must not be null"; + hasShardResponse.set(true); successfulOps.incrementAndGet(); results.consumeResult(result); if (logger.isTraceEnabled()) { @@ -602,8 +605,13 @@ public final ShardSearchRequest buildShardSearchRequest(SearchShardIterator shar String indexName = shardIt.shardId().getIndex().getName(); final String[] routings = indexRoutings.getOrDefault(indexName, Collections.emptySet()) .toArray(new String[0]); - return new ShardSearchRequest(shardIt.getOriginalIndices(), request, shardIt.shardId(), getNumShards(), + ShardSearchRequest shardRequest = new ShardSearchRequest(shardIt.getOriginalIndices(), request, shardIt.shardId(), getNumShards(), filter, indexBoost, timeProvider.getAbsoluteStartMillis(), shardIt.getClusterAlias(), routings); + // if we already received a search result we can inform the shard that it + // can return a null response if the request rewrites to match none rather + // than creating an empty response in the search thread pool. + shardRequest.setMatchNoDocsReturnNullResponse(hasShardResponse.get()); + return shardRequest; } /** diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java index 27b5c9cf3b2a8..06bb29a51ce6c 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java @@ -65,6 +65,7 @@ import java.util.Map; import java.util.function.Function; import java.util.function.IntFunction; +import java.util.stream.Collectors; public final class SearchPhaseController { @@ -427,6 +428,15 @@ private ReducedQueryPhase reducedQueryPhase(Collection res.queryResult().isNull() == false) + .collect(Collectors.toList()); + String errorMsg = "must have at least one non-empty search result, got 0 out of " + total; + assert queryResults.isEmpty() == false : errorMsg; + if (queryResults.isEmpty()) { + throw new IllegalStateException(errorMsg); + } final QuerySearchResult firstResult = queryResults.stream().findFirst().get().queryResult(); final boolean hasSuggest = firstResult.suggest() != null; final boolean hasProfileResults = firstResult.hasProfileResults(); @@ -622,36 +632,38 @@ public void consumeResult(SearchPhaseResult result) { } private synchronized void consumeInternal(QuerySearchResult querySearchResult) { - if (index == bufferSize) { + if (querySearchResult.isNull() == false) { + if (index == bufferSize) { + if (hasAggs) { + ReduceContext reduceContext = controller.reduceContextFunction.apply(false); + InternalAggregations reducedAggs = InternalAggregations.topLevelReduce(Arrays.asList(aggsBuffer), reduceContext); + Arrays.fill(aggsBuffer, null); + aggsBuffer[0] = reducedAggs; + } + if (hasTopDocs) { + TopDocs reducedTopDocs = mergeTopDocs(Arrays.asList(topDocsBuffer), + // we have to merge here in the same way we collect on a shard + querySearchResult.from() + querySearchResult.size(), 0); + Arrays.fill(topDocsBuffer, null); + topDocsBuffer[0] = reducedTopDocs; + } + numReducePhases++; + index = 1; + if (hasAggs) { + progressListener.notifyPartialReduce(progressListener.searchShards(processedShards), + topDocsStats.getTotalHits(), aggsBuffer[0], numReducePhases); + } + } + final int i = index++; if (hasAggs) { - ReduceContext reduceContext = controller.reduceContextFunction.apply(false); - InternalAggregations reducedAggs = InternalAggregations.topLevelReduce(Arrays.asList(aggsBuffer), reduceContext); - Arrays.fill(aggsBuffer, null); - aggsBuffer[0] = reducedAggs; + aggsBuffer[i] = (InternalAggregations) querySearchResult.consumeAggs(); } if (hasTopDocs) { - TopDocs reducedTopDocs = mergeTopDocs(Arrays.asList(topDocsBuffer), - // we have to merge here in the same way we collect on a shard - querySearchResult.from() + querySearchResult.size(), 0); - Arrays.fill(topDocsBuffer, null); - topDocsBuffer[0] = reducedTopDocs; + final TopDocsAndMaxScore topDocs = querySearchResult.consumeTopDocs(); // can't be null + topDocsStats.add(topDocs, querySearchResult.searchTimedOut(), querySearchResult.terminatedEarly()); + setShardIndex(topDocs.topDocs, querySearchResult.getShardIndex()); + topDocsBuffer[i] = topDocs.topDocs; } - numReducePhases++; - index = 1; - if (hasAggs) { - progressListener.notifyPartialReduce(progressListener.searchShards(processedShards), - topDocsStats.getTotalHits(), aggsBuffer[0], numReducePhases); - } - } - final int i = index++; - if (hasAggs) { - aggsBuffer[i] = (InternalAggregations) querySearchResult.consumeAggs(); - } - if (hasTopDocs) { - final TopDocsAndMaxScore topDocs = querySearchResult.consumeTopDocs(); // can't be null - topDocsStats.add(topDocs, querySearchResult.searchTimedOut(), querySearchResult.terminatedEarly()); - setShardIndex(topDocs.topDocs, querySearchResult.getShardIndex()); - topDocsBuffer[i] = topDocs.topDocs; } processedShards[querySearchResult.getShardIndex()] = querySearchResult.getSearchShardTarget(); } @@ -731,7 +743,7 @@ ReducedQueryPhase reduce() { static final class TopDocsStats { final int trackTotalHitsUpTo; - private long totalHits; + long totalHits; private TotalHits.Relation totalHitsRelation; long fetchHits; private float maxScore = Float.NEGATIVE_INFINITY; diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 759feb1d7c68e..7ab04ab6d48b3 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1210,6 +1210,13 @@ private Engine.Searcher acquireSearcher(String source, Engine.SearcherScope scop markSearcherAccessed(); final Engine engine = getEngine(); final Engine.Searcher searcher = engine.acquireSearcher(source, scope); + return wrapSearcher(searcher); + } + + /** + * Wraps the provided searcher acquired with {@link #acquireSearcherNoWrap(String)}. + */ + public Engine.Searcher wrapSearcher(Engine.Searcher searcher) { assert ElasticsearchDirectoryReader.unwrap(searcher.getDirectoryReader()) != null : "DirectoryReader must be an instance or ElasticsearchDirectoryReader"; boolean success = false; diff --git a/server/src/main/java/org/elasticsearch/search/SearchService.java b/server/src/main/java/org/elasticsearch/search/SearchService.java index 84c4903dfd838..ffca1b96ab630 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchService.java +++ b/server/src/main/java/org/elasticsearch/search/SearchService.java @@ -305,11 +305,31 @@ protected void doClose() { } public void executeDfsPhase(ShardSearchRequest request, SearchShardTask task, ActionListener listener) { - rewriteShardRequest(request, ActionListener.map(listener, r -> executeDfsPhase(r, task))); + rewriteShardRequest(request, ActionListener.wrap( + // fork the execution in the search thread pool and wraps the searcher + // to execute the query + context -> { + try { + context.wrapSearcher().execute(() -> { + final SearchPhaseResult result; + try { + result = executeDfsPhase(context, task); + } catch (Exception exc) { + listener.onFailure(exc); + return; + } + listener.onResponse(result); + }); + } catch (Exception exc) { + // if the execution is rejected we need to close the searcher + IOUtils.closeWhileHandlingException(context.searcher); + listener.onFailure(exc); + } + }, listener::onFailure)); } - private DfsSearchResult executeDfsPhase(ShardSearchRequest request, SearchShardTask task) throws IOException { - final SearchContext context = createAndPutContext(request); + private DfsSearchResult executeDfsPhase(SearchRewriteContext rewriteContext, SearchShardTask task) throws IOException { + final SearchContext context = createAndPutContext(rewriteContext); context.incRef(); try { context.setTask(task); @@ -340,15 +360,59 @@ private void loadOrExecuteQueryPhase(final ShardSearchRequest request, final Sea } public void executeQueryPhase(ShardSearchRequest request, SearchShardTask task, ActionListener listener) { - rewriteShardRequest(request, ActionListener.map(listener, r -> executeQueryPhase(r, task))); + assert request.isMatchNoDocsReturnNullResponse() == false || request.numberOfShards() > 1 + : "empty responses require more than one shard"; + rewriteShardRequest(request, ActionListener.wrap( + context -> { + try { + ShardSearchRequest rewritten = context.request; + if (rewritten.isMatchNoDocsReturnNullResponse() + && canRewriteToMatchNone(rewritten.source()) + && rewritten.source().query() instanceof MatchNoneQueryBuilder) { + onMatchNoDocs(context, listener); + } else { + // fork the execution in the search thread pool and wraps the searcher + // to execute the query + context.wrapSearcher().execute(() -> { + final SearchPhaseResult result; + try { + result = executeQueryPhase(context, task); + } catch (Exception exc) { + listener.onFailure(exc); + return; + } + listener.onResponse(result); + }); + } + } catch (Exception exc) { + // if the execution is rejected we need to close the searcher + IOUtils.closeWhileHandlingException(context.searcher); + listener.onFailure(exc); + } + }, listener::onFailure)); + } + + private void onMatchNoDocs(SearchRewriteContext rewriteContext, ActionListener listener) { + // creates a lightweight search context that we use to inform context listeners + // before closing + SearchContext searchContext = createSearchContext(rewriteContext, defaultSearchTimeout); + try (searchContext) { + onNewContext(searchContext); + onFreeContext(searchContext); + } catch (Exception exc) { + listener.onFailure(exc); + return; + } + listener.onResponse(QuerySearchResult.nullInstance()); } private void runAsync(long id, Supplier executable, ActionListener listener) { getExecutor(id).execute(ActionRunnable.supply(listener, executable::get)); } - private SearchPhaseResult executeQueryPhase(ShardSearchRequest request, SearchShardTask task) throws Exception { - final SearchContext context = createAndPutContext(request); + private SearchPhaseResult executeQueryPhase(SearchRewriteContext rewriteContext, SearchShardTask task) throws Exception { + final SearchContext context = createAndPutContext(rewriteContext); + final ShardSearchRequest request = rewriteContext.request; context.incRef(); try { context.setTask(task); @@ -539,15 +603,8 @@ private SearchContext findContext(long id, TransportRequest request) throws Sear } } - final SearchContext createAndPutContext(ShardSearchRequest request) throws IOException { - if (request.scroll() != null && openScrollContexts.get() >= maxOpenScrollContext) { - throw new ElasticsearchException( - "Trying to create too many scroll contexts. Must be less than or equal to: [" + - maxOpenScrollContext + "]. " + "This limit can be set by changing the [" - + MAX_OPEN_SCROLL_CONTEXT.getKey() + "] setting."); - } - - SearchContext context = createContext(request); + final SearchContext createAndPutContext(SearchRewriteContext rewriteContext) { + SearchContext context = createContext(rewriteContext); onNewContext(context); boolean success = false; try { @@ -581,9 +638,16 @@ private void onNewContext(SearchContext context) { } } - final SearchContext createContext(ShardSearchRequest request) throws IOException { - final DefaultSearchContext context = createSearchContext(request, defaultSearchTimeout); + final SearchContext createContext(SearchRewriteContext rewriteContext) { + final DefaultSearchContext context = createSearchContext(rewriteContext, defaultSearchTimeout); try { + if (rewriteContext.request != null && openScrollContexts.get() >= maxOpenScrollContext) { + throw new ElasticsearchException( + "Trying to create too many scroll contexts. Must be less than or equal to: [" + + maxOpenScrollContext + "]. " + "This limit can be set by changing the [" + + MAX_OPEN_SCROLL_CONTEXT.getKey() + "] setting."); + } + final ShardSearchRequest request = rewriteContext.request; if (request.scroll() != null) { context.scrollContext(new ScrollContext()); context.scrollContext().scroll = request.scroll(); @@ -619,41 +683,32 @@ final SearchContext createContext(ShardSearchRequest request) throws IOException } public DefaultSearchContext createSearchContext(ShardSearchRequest request, TimeValue timeout) throws IOException { - return createSearchContext(request, timeout, true, "search"); + IndexShard shard = indicesService.indexServiceSafe(request.shardId().getIndex()).getShard(request.shardId().id()); + SearchRewriteContext rewriteContext = acquireSearcherAndRewrite(request, shard); + // make sure that we wrap the searcher when executing the query + return createSearchContext(rewriteContext.wrapSearcher(), timeout); } - private DefaultSearchContext createSearchContext(ShardSearchRequest request, TimeValue timeout, - boolean assertAsyncActions, String source) - throws IOException { + private DefaultSearchContext createSearchContext(SearchRewriteContext rewriteContext, TimeValue timeout) { + final ShardSearchRequest request = rewriteContext.request; + final Engine.Searcher searcher = rewriteContext.searcher; IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex()); IndexShard indexShard = indexService.getShard(request.shardId().getId()); SearchShardTarget shardTarget = new SearchShardTarget(clusterService.localNode().getId(), indexShard.shardId(), request.getClusterAlias(), OriginalIndices.NONE); - Engine.Searcher searcher = indexShard.acquireSearcher(source); - boolean success = false; - DefaultSearchContext searchContext = null; try { - searchContext = new DefaultSearchContext(idGenerator.incrementAndGet(), request, shardTarget, + DefaultSearchContext searchContext = new DefaultSearchContext(idGenerator.incrementAndGet(), request, shardTarget, searcher, clusterService, indexService, indexShard, bigArrays, threadPool::relativeTimeInMillis, timeout, fetchPhase); - // we clone the query shard context here just for rewriting otherwise we - // might end up with incorrect state since we are using now() or script services - // during rewrite and normalized / evaluate templates etc. - QueryShardContext context = new QueryShardContext(searchContext.getQueryShardContext()); - Rewriteable.rewrite(request.getRewriteable(), context, assertAsyncActions); - assert searchContext.getQueryShardContext().isCacheable(); success = true; + return searchContext; } finally { if (success == false) { - IOUtils.closeWhileHandlingException(searchContext); - if (searchContext == null) { - // we handle the case where the DefaultSearchContext constructor throws an exception since we would otherwise - // leak a searcher and this can have severe implications (unable to obtain shard lock exceptions). - IOUtils.closeWhileHandlingException(searcher); - } + // we handle the case where the DefaultSearchContext constructor throws an exception since we would otherwise + // leak a searcher and this can have severe implications (unable to obtain shard lock exceptions). + IOUtils.closeWhileHandlingException(rewriteContext.searcher); } } - return searchContext; } private void freeAllContextForIndex(Index index) { @@ -1059,24 +1114,49 @@ public static boolean canRewriteToMatchNone(SearchSourceBuilder source) { return aggregations == null || aggregations.mustVisitAllDocs() == false; } - /* * Rewrites the search request with a light weight rewrite context in order to fetch resources asynchronously - * The action listener is guaranteed to be executed on the search thread-pool + * and then rewrites with a searcher when the shard is active. + * The provided action listener is executed on the same thread or in a listener threadpool. */ - private void rewriteShardRequest(ShardSearchRequest request, ActionListener listener) { + private void rewriteShardRequest(ShardSearchRequest request, ActionListener listener) { IndexShard shard = indicesService.indexServiceSafe(request.shardId().getIndex()).getShard(request.shardId().id()); - Executor executor = getExecutor(shard); ActionListener actionListener = ActionListener.wrap(r -> // now we need to check if there is a pending refresh and register - shard.awaitShardSearchActive(b -> executor.execute(ActionRunnable.supply(listener, () -> request))), - listener::onFailure); + shard.awaitShardSearchActive(b -> { + try { + SearchRewriteContext rewriteContext = acquireSearcherAndRewrite(request, shard); + listener.onResponse(rewriteContext); + } catch (Exception e) { + listener.onFailure(e); + } + }), listener::onFailure); // we also do rewrite on the coordinating node (TransportSearchService) but we also need to do it here for BWC as well as // AliasFilters that might need to be rewritten. These are edge-cases but we are every efficient doing the rewrite here so it's not // adding a lot of overhead Rewriteable.rewriteAndFetch(request.getRewriteable(), indicesService.getRewriteContext(request::nowInMillis), actionListener); } + SearchRewriteContext acquireSearcherAndRewrite(ShardSearchRequest request, IndexShard shard) throws IOException { + // acquire the searcher for rewrite with no wrapping in order to avoid costly + // operations. We'll wrap the searcher at a later stage (when executing the query). + Engine.Searcher searcher = shard.acquireSearcherNoWrap("search"); + boolean success = false; + try { + IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex()); + QueryShardContext context = indexService.newQueryShardContext(request.shardId().id(), searcher, + request::nowInMillis, request.getClusterAlias()); + Rewriteable.rewrite(request.getRewriteable(), context, true); + SearchRewriteContext rewrite = new SearchRewriteContext(request, shard, searcher, getExecutor(shard)); + success = true; + return rewrite; + } finally { + if (success == false) { + IOUtils.closeWhileHandlingException(searcher); + } + } + } + /** * Returns a new {@link QueryRewriteContext} with the given {@code now} provider */ @@ -1093,6 +1173,37 @@ public InternalAggregation.ReduceContext createReduceContext(boolean finalReduce finalReduce ? multiBucketConsumerService.create() : bucketCount -> {}, finalReduce); } + static class SearchRewriteContext { + private final ShardSearchRequest request; + private final IndexShard shard; + private Engine.Searcher searcher; + private final Executor executor; + + private boolean isWrapped; + + private SearchRewriteContext(ShardSearchRequest request, + IndexShard shard, + Engine.Searcher searcher, + Executor executor) { + this.request = request; + this.shard = shard; + this.searcher = searcher; + this.executor = executor; + } + + SearchRewriteContext wrapSearcher() { + assert isWrapped == false : "searcher already wrapped"; + isWrapped = true; + searcher = shard.wrapSearcher(searcher); + return this; + } + + void execute(Runnable runnable) { + assert isWrapped : "searcher is not wrapped"; + executor.execute(runnable); + } + } + public static final class CanMatchResponse extends SearchPhaseResult { private final boolean canMatch; private final MinAndMax minAndMax; diff --git a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java index 27543a2285d32..47b10f48e0d99 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java +++ b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java @@ -47,6 +47,7 @@ import org.elasticsearch.indices.InvalidAliasNameException; import org.elasticsearch.search.Scroll; import org.elasticsearch.search.builder.SearchSourceBuilder; +import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskId; import org.elasticsearch.transport.TransportRequest; @@ -75,7 +76,9 @@ public class ShardSearchRequest extends TransportRequest implements IndicesReque private final String preference; private final OriginalIndices originalIndices; - //these are the only two mutable fields, as they are subject to rewriting + private boolean matchNoDocsReturnNullResponse; + + //these are the only mutable fields, as they are subject to rewriting private AliasFilter aliasFilter; private SearchSourceBuilder source; @@ -167,6 +170,11 @@ public ShardSearchRequest(StreamInput in) throws IOException { allowPartialSearchResults = in.readBoolean(); indexRoutings = in.readStringArray(); preference = in.readOptionalString(); + if (in.getVersion().onOrAfter(Version.CURRENT)) { + matchNoDocsReturnNullResponse = in.readBoolean(); + } else { + matchNoDocsReturnNullResponse = true; + } originalIndices = OriginalIndices.readOriginalIndices(in); } @@ -201,6 +209,9 @@ protected final void innerWriteTo(StreamOutput out, boolean asKey) throws IOExce out.writeStringArray(indexRoutings); out.writeOptionalString(preference); } + if (out.getVersion().onOrAfter(Version.CURRENT)) { + out.writeBoolean(matchNoDocsReturnNullResponse); + } } @Override @@ -275,6 +286,19 @@ public String preference() { return preference; } + /** + * Returns true if the caller can handle null response {@link QuerySearchResult#nullInstance()}. + * Defaults to false since the coordinator node needs at least one shard response to build the global + * response. + */ + public boolean isMatchNoDocsReturnNullResponse() { + return matchNoDocsReturnNullResponse; + } + + public void setMatchNoDocsReturnNullResponse(boolean value) { + this.matchNoDocsReturnNullResponse = value; + } + /** * Returns the cache key for this shard search request, based on its content */ diff --git a/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java b/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java index 3d49c96d56119..3151dadadde6b 100644 --- a/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java +++ b/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java @@ -63,18 +63,56 @@ public final class QuerySearchResult extends SearchPhaseResult { private long serviceTimeEWMA = -1; private int nodeQueueSize = -1; + private final boolean isNull; + public QuerySearchResult() { + this(false); } public QuerySearchResult(StreamInput in) throws IOException { super(in); - long id = in.readLong(); - readFromWithId(id, in); + if (in.getVersion().onOrAfter(Version.V_8_0_0)) { + isNull = in.readBoolean(); + } else { + isNull = false; + } + if (isNull == false) { + long id = in.readLong(); + readFromWithId(id, in); + } } public QuerySearchResult(long id, SearchShardTarget shardTarget) { this.requestId = id; setSearchShardTarget(shardTarget); + isNull = false; + } + + private QuerySearchResult(boolean isNull) { + this.isNull = isNull; + } + + private static final QuerySearchResult nullInstance = new QuerySearchResult(true); + + /** + * Returns an instance that contains no response. + */ + public static QuerySearchResult nullInstance() { + return nullInstance; + } + + /** + * Returns true if the result doesn't contain any useful information. + * It is used by the search action to avoid creating an empty response on + * shard request that rewrites to match_no_docs. + * + * TODO: Currently we need the concrete aggregators to build empty responses. This means that we cannot + * build an empty response in the coordinating node so we rely on this hack to ensure that at least one shard + * returns a valid empty response. We should move the ability to create empty responses to aggregation builders + * in order to allow building empty responses directly from the coordinating node. + */ + public boolean isNull() { + return isNull; } @Override @@ -173,6 +211,10 @@ public void aggregations(InternalAggregations aggregations) { hasAggs = aggregations != null; } + public InternalAggregations aggregations() { + return aggregations; + } + /** * Returns and nulls out the profiled results for this search, or potentially null if result was empty. * This allows to free up memory once the profiled result is consumed. @@ -300,8 +342,13 @@ public void readFromWithId(long id, StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { - out.writeLong(requestId); - writeToNoId(out); + if (out.getVersion().onOrAfter(Version.V_8_0_0)) { + out.writeBoolean(isNull); + } + if (isNull == false) { + out.writeLong(requestId); + writeToNoId(out); + } } public void writeToNoId(StreamOutput out) throws IOException { diff --git a/server/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java b/server/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java index 641d5bf2c59b4..f49d3a69caca0 100644 --- a/server/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java @@ -338,16 +338,37 @@ private static SearchRequest randomSearchRequest() { } public void testConsumer() { + consumerTestCase(0); + } + + public void testConsumerWithEmptyResponse() { + consumerTestCase(randomIntBetween(1, 5)); + } + + private void consumerTestCase(int numEmptyResponses) { + int numShards = 3 + numEmptyResponses; int bufferSize = randomIntBetween(2, 3); SearchRequest request = randomSearchRequest(); request.source(new SearchSourceBuilder().aggregation(AggregationBuilders.avg("foo"))); request.setBatchedReduceSize(bufferSize); - ArraySearchPhaseResults consumer = searchPhaseController.newSearchPhaseResults(NOOP, request, 3); - assertEquals(0, reductions.size()); + ArraySearchPhaseResults consumer = + searchPhaseController.newSearchPhaseResults(NOOP, request, 3+numEmptyResponses); + if (numEmptyResponses == 0) { + assertEquals(0, reductions.size()); + } + if (numEmptyResponses > 0) { + QuerySearchResult empty = QuerySearchResult.nullInstance(); + int shardId = 2 + numEmptyResponses; + empty.setShardIndex(2+numEmptyResponses); + empty.setSearchShardTarget(new SearchShardTarget("node", new ShardId("a", "b", shardId), null, OriginalIndices.NONE)); + consumer.consumeResult(empty); + numEmptyResponses --; + } + QuerySearchResult result = new QuerySearchResult(0, new SearchShardTarget("node", new ShardId("a", "b", 0), null, OriginalIndices.NONE)); result.topDocs(new TopDocsAndMaxScore(new TopDocs(new TotalHits(0, TotalHits.Relation.EQUAL_TO), new ScoreDoc[0]), Float.NaN), - new DocValueFormat[0]); + new DocValueFormat[0]); InternalAggregations aggs = new InternalAggregations(Collections.singletonList(new InternalMax("test", 1.0D, DocValueFormat.RAW, Collections.emptyList(), Collections.emptyMap()))); result.aggregations(aggs); @@ -356,7 +377,7 @@ public void testConsumer() { result = new QuerySearchResult(1, new SearchShardTarget("node", new ShardId("a", "b", 0), null, OriginalIndices.NONE)); result.topDocs(new TopDocsAndMaxScore(new TopDocs(new TotalHits(0, TotalHits.Relation.EQUAL_TO), new ScoreDoc[0]), Float.NaN), - new DocValueFormat[0]); + new DocValueFormat[0]); aggs = new InternalAggregations(Collections.singletonList(new InternalMax("test", 3.0D, DocValueFormat.RAW, Collections.emptyList(), Collections.emptyMap()))); result.aggregations(aggs); @@ -365,20 +386,38 @@ public void testConsumer() { result = new QuerySearchResult(1, new SearchShardTarget("node", new ShardId("a", "b", 0), null, OriginalIndices.NONE)); result.topDocs(new TopDocsAndMaxScore(new TopDocs(new TotalHits(0, TotalHits.Relation.EQUAL_TO), new ScoreDoc[0]), Float.NaN), - new DocValueFormat[0]); + new DocValueFormat[0]); aggs = new InternalAggregations(Collections.singletonList(new InternalMax("test", 2.0D, DocValueFormat.RAW, Collections.emptyList(), Collections.emptyMap()))); result.aggregations(aggs); result.setShardIndex(1); consumer.consumeResult(result); + + while (numEmptyResponses > 0) { + result = QuerySearchResult.nullInstance(); + int shardId = 2 + numEmptyResponses; + result.setShardIndex(shardId); + result.setSearchShardTarget(new SearchShardTarget("node", new ShardId("a", "b", shardId), null, OriginalIndices.NONE)); + consumer.consumeResult(result); + numEmptyResponses--; + + } + final int numTotalReducePhases; - if (bufferSize == 2) { + if (numShards > bufferSize) { assertThat(consumer, instanceOf(SearchPhaseController.QueryPhaseResultConsumer.class)); - assertEquals(1, ((SearchPhaseController.QueryPhaseResultConsumer)consumer).getNumReducePhases()); - assertEquals(2, ((SearchPhaseController.QueryPhaseResultConsumer)consumer).getNumBuffered()); - assertEquals(1, reductions.size()); - assertEquals(false, reductions.get(0)); - numTotalReducePhases = 2; + if (bufferSize == 2) { + assertEquals(1, ((SearchPhaseController.QueryPhaseResultConsumer) consumer).getNumReducePhases()); + assertEquals(2, ((SearchPhaseController.QueryPhaseResultConsumer) consumer).getNumBuffered()); + assertEquals(1, reductions.size()); + assertEquals(false, reductions.get(0)); + numTotalReducePhases = 2; + } else { + assertEquals(0, ((SearchPhaseController.QueryPhaseResultConsumer) consumer).getNumReducePhases()); + assertEquals(3, ((SearchPhaseController.QueryPhaseResultConsumer) consumer).getNumBuffered()); + assertEquals(0, reductions.size()); + numTotalReducePhases = 1; + } } else { assertThat(consumer, not(instanceOf(SearchPhaseController.QueryPhaseResultConsumer.class))); assertEquals(0, reductions.size()); diff --git a/server/src/test/java/org/elasticsearch/indices/IndicesRequestCacheIT.java b/server/src/test/java/org/elasticsearch/indices/IndicesRequestCacheIT.java index 3ea41962e4855..fa83cb2662a8e 100644 --- a/server/src/test/java/org/elasticsearch/indices/IndicesRequestCacheIT.java +++ b/server/src/test/java/org/elasticsearch/indices/IndicesRequestCacheIT.java @@ -29,6 +29,7 @@ import org.elasticsearch.common.time.DateFormatter; import org.elasticsearch.index.cache.request.RequestCacheStats; import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Bucket; @@ -123,20 +124,26 @@ public void testQueryRewrite() throws Exception { assertCacheState(client, "index", 0, 0); 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")).setPreFilterShardSize(Integer.MAX_VALUE).get(); + .setQuery(QueryBuilders.rangeQuery("s").gte("2016-03-19").lte("2016-03-25")) + .addAggregation(new GlobalAggregationBuilder("global")) + .setPreFilterShardSize(Integer.MAX_VALUE).get(); ElasticsearchAssertions.assertAllSuccessful(r1); assertThat(r1.getHits().getTotalHits().value, equalTo(7L)); assertCacheState(client, "index", 0, 5); 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")) + .setQuery(QueryBuilders.rangeQuery("s").gte("2016-03-20").lte("2016-03-26")) + // to ensure that query is executed even if it rewrites to match_no_docs + .addAggregation(new GlobalAggregationBuilder("global")) .setPreFilterShardSize(Integer.MAX_VALUE).get(); ElasticsearchAssertions.assertAllSuccessful(r2); assertThat(r2.getHits().getTotalHits().value, equalTo(7L)); assertCacheState(client, "index", 3, 7); 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")).setPreFilterShardSize(Integer.MAX_VALUE) + .setQuery(QueryBuilders.rangeQuery("s").gte("2016-03-21").lte("2016-03-27")) + .addAggregation(new GlobalAggregationBuilder("global")) + .setPreFilterShardSize(Integer.MAX_VALUE) .get(); ElasticsearchAssertions.assertAllSuccessful(r3); assertThat(r3.getHits().getTotalHits().value, equalTo(7L)); diff --git a/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java b/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java index 5691530dd32fe..75435f40f7dfb 100644 --- a/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java +++ b/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java @@ -66,6 +66,7 @@ import org.elasticsearch.script.MockScriptPlugin; import org.elasticsearch.script.Script; import org.elasticsearch.script.ScriptType; +import org.elasticsearch.search.aggregations.AggregationBuilders; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.MultiBucketConsumerService; import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregationBuilder; @@ -77,6 +78,7 @@ import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.ShardSearchRequest; +import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.search.suggest.SuggestBuilder; import org.elasticsearch.test.ESSingleNodeTestCase; import org.junit.Before; @@ -168,10 +170,12 @@ public void onIndexModule(IndexModule indexModule) { indexModule.addSearchOperationListener(new SearchOperationListener() { @Override public void onNewContext(SearchContext context) { - if ("throttled_threadpool_index".equals(context.indexShard().shardId().getIndex().getName())) { - assertThat(Thread.currentThread().getName(), startsWith("elasticsearch[node_s_0][search_throttled]")); - } else { - assertThat(Thread.currentThread().getName(), startsWith("elasticsearch[node_s_0][search]")); + if (context.query() != null) { + if ("throttled_threadpool_index".equals(context.indexShard().shardId().getIndex().getName())) { + assertThat(Thread.currentThread().getName(), startsWith("elasticsearch[node_s_0][search_throttled]")); + } else { + assertThat(Thread.currentThread().getName(), startsWith("elasticsearch[node_s_0][search]")); + } } } @@ -357,15 +361,11 @@ public void testTimeout() throws IOException { final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); final IndexShard indexShard = indexService.getShard(0); SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true); - final SearchContext contextWithDefaultTimeout = service.createContext( - new ShardSearchRequest( - OriginalIndices.NONE, - searchRequest, - indexShard.shardId(), - 1, - new AliasFilter(null, Strings.EMPTY_ARRAY), - 1.0f, -1, null, null) - ); + SearchService.SearchRewriteContext rewriteContext = service.acquireSearcherAndRewrite( + new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 1, + new AliasFilter(null, Strings.EMPTY_ARRAY), 1.0f, -1, null, null), + indexShard); + final SearchContext contextWithDefaultTimeout = service.createContext(rewriteContext); try { // the search context should inherit the default timeout assertThat(contextWithDefaultTimeout.timeout(), equalTo(TimeValue.timeValueSeconds(5))); @@ -376,15 +376,11 @@ public void testTimeout() throws IOException { final long seconds = randomIntBetween(6, 10); searchRequest.source(new SearchSourceBuilder().timeout(TimeValue.timeValueSeconds(seconds))); - final SearchContext context = service.createContext( - new ShardSearchRequest( - OriginalIndices.NONE, - searchRequest, - indexShard.shardId(), - 1, - new AliasFilter(null, Strings.EMPTY_ARRAY), - 1.0f, -1, null, null) - ); + rewriteContext = service.acquireSearcherAndRewrite( + new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 1, + new AliasFilter(null, Strings.EMPTY_ARRAY), 1.0f, -1, null, null), + indexShard); + final SearchContext context = service.createContext(rewriteContext); try { // the search context should inherit the query timeout assertThat(context.timeout(), equalTo(TimeValue.timeValueSeconds(seconds))); @@ -412,19 +408,25 @@ public void testMaxDocvalueFieldsSearch() throws IOException { for (int i = 0; i < indexService.getIndexSettings().getMaxDocvalueFields(); i++) { searchSourceBuilder.docValueField("field" + i); } - try (SearchContext context = service.createContext( - new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 1, - new AliasFilter(null, Strings.EMPTY_ARRAY), 1.0f, -1, null, null)) - ) { - assertNotNull(context); + + ShardSearchRequest shardRequest = new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 1, + new AliasFilter(null, Strings.EMPTY_ARRAY), 1.0f, -1, null, null); + + { + SearchService.SearchRewriteContext rewriteContext = service.acquireSearcherAndRewrite(shardRequest, indexShard); + try (SearchContext context = service.createContext(rewriteContext)) { + assertNotNull(context); + } + } + + { + SearchService.SearchRewriteContext rewriteContext = service.acquireSearcherAndRewrite(shardRequest, indexShard); searchSourceBuilder.docValueField("one_field_too_much"); IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, - () -> service.createContext(new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 1, - new AliasFilter(null, Strings.EMPTY_ARRAY), 1.0f, -1, null, null))); + () -> service.createContext(rewriteContext)); assertEquals( - "Trying to retrieve too many docvalue_fields. Must be less than or equal to: [100] but was [101]. " - + "This limit can be set by changing the [index.max_docvalue_fields_search] index level setting.", - ex.getMessage()); + "Trying to retrieve too many docvalue_fields. Must be less than or equal to: [100] but was [101]. " + + "This limit can be set by changing the [index.max_docvalue_fields_search] index level setting.", ex.getMessage()); } } @@ -447,20 +449,28 @@ public void testMaxScriptFieldsSearch() throws IOException { searchSourceBuilder.scriptField("field" + i, new Script(ScriptType.INLINE, MockScriptEngine.NAME, CustomScriptPlugin.DUMMY_SCRIPT, Collections.emptyMap())); } - try (SearchContext context = service.createContext(new ShardSearchRequest(OriginalIndices.NONE, searchRequest, - indexShard.shardId(), 1, new AliasFilter(null, Strings.EMPTY_ARRAY), - 1.0f, -1, null, null))) { - assertNotNull(context); + + ShardSearchRequest shardRequest = new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 1, + new AliasFilter(null, Strings.EMPTY_ARRAY), 1.0f, -1, null, null); + + { + SearchService.SearchRewriteContext rewriteContext = service.acquireSearcherAndRewrite(shardRequest, indexShard); + try (SearchContext context = service.createContext(rewriteContext)) { + assertNotNull(context); + } + } + + { searchSourceBuilder.scriptField("anotherScriptField", - new Script(ScriptType.INLINE, MockScriptEngine.NAME, CustomScriptPlugin.DUMMY_SCRIPT, Collections.emptyMap())); + new Script(ScriptType.INLINE, MockScriptEngine.NAME, CustomScriptPlugin.DUMMY_SCRIPT, Collections.emptyMap())); + SearchService.SearchRewriteContext rewriteContext = service.acquireSearcherAndRewrite(shardRequest, indexShard); IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, - () -> service.createContext(new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 1, - new AliasFilter(null, Strings.EMPTY_ARRAY), 1.0f, -1, null, null))); + () -> service.createContext(rewriteContext)); assertEquals( - "Trying to retrieve too many script_fields. Must be less than or equal to: [" + maxScriptFields + "] but was [" - + (maxScriptFields + 1) - + "]. This limit can be set by changing the [index.max_script_fields] index level setting.", - ex.getMessage()); + "Trying to retrieve too many script_fields. Must be less than or equal to: [" + maxScriptFields + "] but was [" + + (maxScriptFields + 1) + + "]. This limit can be set by changing the [index.max_script_fields] index level setting.", + ex.getMessage()); } } @@ -477,17 +487,19 @@ public void testIgnoreScriptfieldIfSizeZero() throws IOException { searchSourceBuilder.scriptField("field" + 0, new Script(ScriptType.INLINE, MockScriptEngine.NAME, CustomScriptPlugin.DUMMY_SCRIPT, Collections.emptyMap())); searchSourceBuilder.size(0); - try (SearchContext context = service.createContext(new ShardSearchRequest(OriginalIndices.NONE, - searchRequest, indexShard.shardId(), 1, new AliasFilter(null, Strings.EMPTY_ARRAY), - 1.0f, -1, null, null))) { - assertEquals(0, context.scriptFields().fields().size()); + SearchService.SearchRewriteContext rewriteContext = service.acquireSearcherAndRewrite( + new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 1, + new AliasFilter(null, Strings.EMPTY_ARRAY), 1.0f, -1, null, null), + indexShard); + try (SearchContext context = service.createContext(rewriteContext)) { + assertEquals(0, context.scriptFields().fields().size()); } } /** * test that creating more than the allowed number of scroll contexts throws an exception */ - public void testMaxOpenScrollContexts() throws RuntimeException { + public void testMaxOpenScrollContexts() throws RuntimeException, IOException { createIndex("index"); client().prepareIndex("index").setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); @@ -513,8 +525,10 @@ public void testMaxOpenScrollContexts() throws RuntimeException { client().prepareSearch("index").setSize(1).setScroll("1m").get(); } + SearchService.SearchRewriteContext rewriteContext = + service.acquireSearcherAndRewrite(new ShardScrollRequestTest(indexShard.shardId()), indexShard); ElasticsearchException ex = expectThrows(ElasticsearchException.class, - () -> service.createAndPutContext(new ShardScrollRequestTest(indexShard.shardId()))); + () -> service.createAndPutContext(rewriteContext)); assertEquals( "Trying to create too many scroll contexts. Must be less than or equal to: [" + SearchService.MAX_OPEN_SCROLL_CONTEXT.get(Settings.EMPTY) + "]. " + @@ -592,7 +606,7 @@ public Scroll scroll() { } } - public void testCanMatch() throws IOException { + public void testCanMatch() throws IOException, InterruptedException { createIndex("index"); final SearchService service = getInstanceFromNode(SearchService.class); final IndicesService indicesService = getInstanceFromNode(IndicesService.class); @@ -625,11 +639,32 @@ public void testCanMatch() throws IOException { new AliasFilter(null, Strings.EMPTY_ARRAY), 1f, -1, null, null)).canMatch()); assertEquals(numWrapReader, numWrapInvocations.get()); - // make sure that the wrapper is called when the context is actually created - service.createContext(new ShardSearchRequest(OriginalIndices.NONE, searchRequest, - indexShard.shardId(), 1, new AliasFilter(null, Strings.EMPTY_ARRAY), - 1f, -1, null, null)).close(); - assertEquals(numWrapReader+1, numWrapInvocations.get()); + ShardSearchRequest request = new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 1, + new AliasFilter(null, Strings.EMPTY_ARRAY), 1.0f, -1, null, null); + + CountDownLatch latch = new CountDownLatch(1); + SearchShardTask task = new SearchShardTask(123L, "", "", "", null, Collections.emptyMap()); + service.executeQueryPhase(request, task, new ActionListener() { + @Override + public void onResponse(SearchPhaseResult searchPhaseResult) { + try { + // make sure that the wrapper is called when the query is actually executed + assertEquals(numWrapReader+1, numWrapInvocations.get()); + } finally { + latch.countDown(); + } + } + + @Override + public void onFailure(Exception e) { + try { + throw new AssertionError(e); + } finally { + latch.countDown(); + } + } + }); + latch.await(); } public void testCanRewriteToMatchNone() { @@ -744,18 +779,123 @@ public void testCreateSearchContextFailure() throws IOException { final IndexService indexService = createIndex(index); final SearchService service = getInstanceFromNode(SearchService.class); final ShardId shardId = new ShardId(indexService.index(), 0); + IndexShard indexShard = indexService.getShard(0); - NullPointerException e = expectThrows(NullPointerException.class, - () -> service.createContext( - new ShardSearchRequest(shardId, 0, null) { - @Override - public SearchType searchType() { - // induce an artificial NPE - throw new NullPointerException("expected"); - } + SearchService.SearchRewriteContext rewriteContext = service.acquireSearcherAndRewrite( + new ShardSearchRequest(shardId, 0, AliasFilter.EMPTY) { + @Override + public SearchType searchType() { + // induce an artificial NPE + throw new NullPointerException("expected"); } - )); + }, indexShard); + NullPointerException e = expectThrows(NullPointerException.class, + () -> service.createContext(rewriteContext)); assertEquals("expected", e.getMessage()); assertEquals("should have 2 store refs (IndexService + InternalEngine)", 2, indexService.getShard(0).store().refCount()); } + + public void testMatchNoDocsEmptyResponse() throws InterruptedException { + createIndex("index"); + Thread currentThread = Thread.currentThread(); + SearchService service = getInstanceFromNode(SearchService.class); + IndicesService indicesService = getInstanceFromNode(IndicesService.class); + IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); + IndexShard indexShard = indexService.getShard(0); + SearchRequest searchRequest = new SearchRequest() + .allowPartialSearchResults(false) + .source(new SearchSourceBuilder() + .aggregation(AggregationBuilders.count("count").field("value"))); + ShardSearchRequest shardRequest = new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), + 5, AliasFilter.EMPTY, 1.0f, 0, null, null); + SearchShardTask task = new SearchShardTask(123L, "", "", "", null, Collections.emptyMap()); + + { + CountDownLatch latch = new CountDownLatch(1); + shardRequest.source().query(new MatchAllQueryBuilder()); + service.executeQueryPhase(shardRequest, task, new ActionListener<>() { + @Override + public void onResponse(SearchPhaseResult result) { + try { + assertNotSame(Thread.currentThread(), currentThread); + assertThat(Thread.currentThread().getName(), startsWith("elasticsearch[node_s_0][search]")); + assertThat(result, instanceOf(QuerySearchResult.class)); + assertFalse(result.queryResult().isNull()); + assertNotNull(result.queryResult().topDocs()); + assertNotNull(result.queryResult().aggregations()); + } finally { + latch.countDown(); + } + } + + @Override + public void onFailure(Exception exc) { + try { + throw new AssertionError(exc); + } finally { + latch.countDown(); + } + } + }); + latch.await(); + } + + { + CountDownLatch latch = new CountDownLatch(1); + shardRequest.source().query(new MatchNoneQueryBuilder()); + service.executeQueryPhase(shardRequest, task, new ActionListener<>() { + @Override + public void onResponse(SearchPhaseResult result) { + try { + assertNotSame(Thread.currentThread(), currentThread); + assertThat(Thread.currentThread().getName(), startsWith("elasticsearch[node_s_0][search]")); + assertThat(result, instanceOf(QuerySearchResult.class)); + assertFalse(result.queryResult().isNull()); + assertNotNull(result.queryResult().topDocs()); + assertNotNull(result.queryResult().aggregations()); + } finally { + latch.countDown(); + } + } + + @Override + public void onFailure(Exception exc) { + try { + throw new AssertionError(exc); + } finally { + latch.countDown(); + } + } + }); + latch.await(); + } + + { + CountDownLatch latch = new CountDownLatch(1); + shardRequest.setMatchNoDocsReturnNullResponse(true); + service.executeQueryPhase(shardRequest, task, new ActionListener<>() { + @Override + public void onResponse(SearchPhaseResult result) { + try { + // make sure we don't use the search threadpool + assertSame(Thread.currentThread(), currentThread); + assertThat(result, instanceOf(QuerySearchResult.class)); + assertTrue(result.queryResult().isNull()); + } finally { + latch.countDown(); + } + } + + @Override + public void onFailure(Exception e) { + try { + throw new AssertionError(e); + } finally { + latch.countDown(); + } + } + }); + latch.await(); + } + } } diff --git a/x-pack/plugin/frozen-indices/src/test/java/org/elasticsearch/index/engine/FrozenIndexTests.java b/x-pack/plugin/frozen-indices/src/test/java/org/elasticsearch/index/engine/FrozenIndexTests.java index 3d06431c554b9..f656c47183abf 100644 --- a/x-pack/plugin/frozen-indices/src/test/java/org/elasticsearch/index/engine/FrozenIndexTests.java +++ b/x-pack/plugin/frozen-indices/src/test/java/org/elasticsearch/index/engine/FrozenIndexTests.java @@ -37,6 +37,7 @@ import org.elasticsearch.protocol.xpack.frozen.FreezeRequest; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.search.SearchService; +import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregationBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.ShardSearchRequest; @@ -46,6 +47,7 @@ import org.hamcrest.Matchers; import java.io.IOException; +import java.util.Arrays; import java.util.Collection; import java.util.EnumSet; import java.util.concurrent.CountDownLatch; @@ -122,26 +124,48 @@ public void testSearchAndGetAPIsAreThrottled() throws InterruptedException, IOEx int numRefreshes = 0; for (int i = 0; i < numRequests; i++) { numRefreshes++; - switch (randomIntBetween(0, 3)) { + // make sure that we don't share the frozen reader in + // concurrent requests + CountDownLatch reqLatch = new CountDownLatch(1); + switch (randomFrom(Arrays.asList(0, 1, 2))) { case 0: - client().prepareGet("index", "" + randomIntBetween(0, 9)).execute(ActionListener.wrap(latch::countDown)); + client().prepareGet("index", "" + randomIntBetween(0, 9)) + .execute(ActionListener.wrap(() -> { + latch.countDown(); + reqLatch.countDown(); + })); + reqLatch.await(); break; case 1: client().prepareSearch("index").setIndicesOptions(IndicesOptions.STRICT_EXPAND_OPEN_FORBID_CLOSED) .setSearchType(SearchType.QUERY_THEN_FETCH) - .execute(ActionListener.wrap(latch::countDown)); + .execute(ActionListener.wrap(() -> { + latch.countDown(); + reqLatch.countDown(); + })); // in total 4 refreshes 1x query & 1x fetch per shard (we have 2) numRefreshes += 3; + reqLatch.await(); break; case 2: - client().prepareTermVectors("index", "" + randomIntBetween(0, 9)).execute(ActionListener.wrap(latch::countDown)); + client().prepareTermVectors("index", "" + randomIntBetween(0, 9)) + .execute(ActionListener.wrap(() -> { + latch.countDown(); + reqLatch.countDown(); + })); + reqLatch.await(); break; case 3: client().prepareExplain("index", "" + randomIntBetween(0, 9)).setQuery(new MatchAllQueryBuilder()) - .execute(ActionListener.wrap(latch::countDown)); + .execute(ActionListener.wrap(() -> { + latch.countDown(); + reqLatch.countDown(); + })); + reqLatch.await(); break; - default: - assert false; + + default: + assert false; } } latch.await(); From 534b5525ed54f1e4efbe6267bb154b9be578a6f7 Mon Sep 17 00:00:00 2001 From: jimczi Date: Thu, 30 Jan 2020 23:03:28 +0100 Subject: [PATCH 02/37] add serialization test --- .../search/internal/ShardSearchRequestTests.java | 6 +++++- .../elasticsearch/search/query/QuerySearchResultTests.java | 7 +++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java b/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java index c913bbe4b9a37..2e2384f0f84fd 100644 --- a/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java +++ b/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java @@ -75,6 +75,8 @@ public void testSerialization() throws Exception { assertEquals(deserializedRequest.indexBoost(), shardSearchTransportRequest.indexBoost(), 0.0f); assertEquals(deserializedRequest.getClusterAlias(), shardSearchTransportRequest.getClusterAlias()); assertEquals(shardSearchTransportRequest.allowPartialSearchResults(), deserializedRequest.allowPartialSearchResults()); + assertEquals(deserializedRequest.isMatchNoDocsReturnNullResponse(), + shardSearchTransportRequest.isMatchNoDocsReturnNullResponse()); } private ShardSearchRequest createShardSearchRequest() throws IOException { @@ -88,9 +90,11 @@ private ShardSearchRequest createShardSearchRequest() throws IOException { filteringAliases = new AliasFilter(null, Strings.EMPTY_ARRAY); } final String[] routings = generateRandomStringArray(5, 10, false, true); - return new ShardSearchRequest(new OriginalIndices(searchRequest), searchRequest, shardId, + ShardSearchRequest req = new ShardSearchRequest(new OriginalIndices(searchRequest), searchRequest, shardId, randomIntBetween(1, 100), filteringAliases, randomBoolean() ? 1.0f : randomFloat(), Math.abs(randomLong()), randomAlphaOfLengthBetween(3, 10), routings); + req.setMatchNoDocsReturnNullResponse(randomBoolean()); + return req; } public void testFilteringAliases() throws Exception { diff --git a/server/src/test/java/org/elasticsearch/search/query/QuerySearchResultTests.java b/server/src/test/java/org/elasticsearch/search/query/QuerySearchResultTests.java index ef49d8e436e3e..c67850436b393 100644 --- a/server/src/test/java/org/elasticsearch/search/query/QuerySearchResultTests.java +++ b/server/src/test/java/org/elasticsearch/search/query/QuerySearchResultTests.java @@ -99,4 +99,11 @@ public void testSerialization() throws Exception { } assertEquals(querySearchResult.terminatedEarly(), deserialized.terminatedEarly()); } + + public void testNullResponse() throws Exception { + QuerySearchResult querySearchResult = QuerySearchResult.nullInstance(); + QuerySearchResult deserialized = + copyWriteable(querySearchResult, namedWriteableRegistry, QuerySearchResult::new, Version.CURRENT); + assertEquals(querySearchResult.isNull(), deserialized.isNull()); + } } From c02f352fe574de396eef1c89f98de74314a35133 Mon Sep 17 00:00:00 2001 From: jimczi Date: Thu, 30 Jan 2020 23:40:32 +0100 Subject: [PATCH 03/37] iter --- .../java/org/elasticsearch/indices/IndicesRequestCacheIT.java | 2 +- .../java/org/elasticsearch/index/engine/FrozenIndexTests.java | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/indices/IndicesRequestCacheIT.java b/server/src/test/java/org/elasticsearch/indices/IndicesRequestCacheIT.java index fa83cb2662a8e..ca660aa1a2134 100644 --- a/server/src/test/java/org/elasticsearch/indices/IndicesRequestCacheIT.java +++ b/server/src/test/java/org/elasticsearch/indices/IndicesRequestCacheIT.java @@ -125,6 +125,7 @@ public void testQueryRewrite() throws Exception { 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")) + // to ensure that query is executed even if it rewrites to match_no_docs .addAggregation(new GlobalAggregationBuilder("global")) .setPreFilterShardSize(Integer.MAX_VALUE).get(); ElasticsearchAssertions.assertAllSuccessful(r1); @@ -133,7 +134,6 @@ 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")) - // to ensure that query is executed even if it rewrites to match_no_docs .addAggregation(new GlobalAggregationBuilder("global")) .setPreFilterShardSize(Integer.MAX_VALUE).get(); ElasticsearchAssertions.assertAllSuccessful(r2); diff --git a/x-pack/plugin/frozen-indices/src/test/java/org/elasticsearch/index/engine/FrozenIndexTests.java b/x-pack/plugin/frozen-indices/src/test/java/org/elasticsearch/index/engine/FrozenIndexTests.java index f656c47183abf..3a976a2cd14b8 100644 --- a/x-pack/plugin/frozen-indices/src/test/java/org/elasticsearch/index/engine/FrozenIndexTests.java +++ b/x-pack/plugin/frozen-indices/src/test/java/org/elasticsearch/index/engine/FrozenIndexTests.java @@ -37,7 +37,6 @@ import org.elasticsearch.protocol.xpack.frozen.FreezeRequest; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.search.SearchService; -import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregationBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.ShardSearchRequest; From f5684ecfc7c42588d167d0eec759f9f6f6a1ceb9 Mon Sep 17 00:00:00 2001 From: jimczi Date: Fri, 31 Jan 2020 00:35:04 +0100 Subject: [PATCH 04/37] fix bwc issue --- .../elasticsearch/search/internal/ShardSearchRequest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java index 47b10f48e0d99..340e8e5295c4c 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java +++ b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java @@ -170,10 +170,10 @@ public ShardSearchRequest(StreamInput in) throws IOException { allowPartialSearchResults = in.readBoolean(); indexRoutings = in.readStringArray(); preference = in.readOptionalString(); - if (in.getVersion().onOrAfter(Version.CURRENT)) { + if (in.getVersion().onOrAfter(Version.V_8_0_0)) { matchNoDocsReturnNullResponse = in.readBoolean(); } else { - matchNoDocsReturnNullResponse = true; + matchNoDocsReturnNullResponse = false; } originalIndices = OriginalIndices.readOriginalIndices(in); } @@ -209,7 +209,7 @@ protected final void innerWriteTo(StreamOutput out, boolean asKey) throws IOExce out.writeStringArray(indexRoutings); out.writeOptionalString(preference); } - if (out.getVersion().onOrAfter(Version.CURRENT)) { + if (out.getVersion().onOrAfter(Version.V_8_0_0)) { out.writeBoolean(matchNoDocsReturnNullResponse); } } From 0acf244b16fc4a7e9bbfeb64160b71d1ae1a7535 Mon Sep 17 00:00:00 2001 From: jimczi Date: Mon, 3 Feb 2020 11:17:08 +0100 Subject: [PATCH 05/37] address review --- .../action/search/AbstractSearchAsyncAction.java | 4 ++-- .../org/elasticsearch/search/SearchService.java | 4 ++-- .../search/internal/ShardSearchRequest.java | 16 ++++++++-------- 3 files changed, 12 insertions(+), 12 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index bc7b347531ad0..bd1a3bdd3123c 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -467,9 +467,9 @@ public final void onShardFailure(final int shardIndex, @Nullable SearchShardTarg protected void onShardResult(Result result, SearchShardIterator shardIt) { assert result.getShardIndex() != -1 : "shard index is not set"; assert result.getSearchShardTarget() != null : "search shard target must not be null"; - hasShardResponse.set(true); successfulOps.incrementAndGet(); results.consumeResult(result); + hasShardResponse.set(true); if (logger.isTraceEnabled()) { logger.trace("got first-phase result from {}", result != null ? result.getSearchShardTarget() : null); } @@ -610,7 +610,7 @@ public final ShardSearchRequest buildShardSearchRequest(SearchShardIterator shar // if we already received a search result we can inform the shard that it // can return a null response if the request rewrites to match none rather // than creating an empty response in the search thread pool. - shardRequest.setMatchNoDocsReturnNullResponse(hasShardResponse.get()); + shardRequest.canReturnNullResponseIfMatchNoDocs(hasShardResponse.get()); return shardRequest; } diff --git a/server/src/main/java/org/elasticsearch/search/SearchService.java b/server/src/main/java/org/elasticsearch/search/SearchService.java index ffca1b96ab630..2d55e86c2fa3a 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchService.java +++ b/server/src/main/java/org/elasticsearch/search/SearchService.java @@ -360,13 +360,13 @@ private void loadOrExecuteQueryPhase(final ShardSearchRequest request, final Sea } public void executeQueryPhase(ShardSearchRequest request, SearchShardTask task, ActionListener listener) { - assert request.isMatchNoDocsReturnNullResponse() == false || request.numberOfShards() > 1 + assert request.canReturnNullResponseIfMatchNoDocs() == false || request.numberOfShards() > 1 : "empty responses require more than one shard"; rewriteShardRequest(request, ActionListener.wrap( context -> { try { ShardSearchRequest rewritten = context.request; - if (rewritten.isMatchNoDocsReturnNullResponse() + if (rewritten.canReturnNullResponseIfMatchNoDocs() && canRewriteToMatchNone(rewritten.source()) && rewritten.source().query() instanceof MatchNoneQueryBuilder) { onMatchNoDocs(context, listener); diff --git a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java index 340e8e5295c4c..f46be3755cee5 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java +++ b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java @@ -76,7 +76,7 @@ public class ShardSearchRequest extends TransportRequest implements IndicesReque private final String preference; private final OriginalIndices originalIndices; - private boolean matchNoDocsReturnNullResponse; + private boolean canReturnNullResponseIfMatchNoDocs; //these are the only mutable fields, as they are subject to rewriting private AliasFilter aliasFilter; @@ -171,9 +171,9 @@ public ShardSearchRequest(StreamInput in) throws IOException { indexRoutings = in.readStringArray(); preference = in.readOptionalString(); if (in.getVersion().onOrAfter(Version.V_8_0_0)) { - matchNoDocsReturnNullResponse = in.readBoolean(); + canReturnNullResponseIfMatchNoDocs = in.readBoolean(); } else { - matchNoDocsReturnNullResponse = false; + canReturnNullResponseIfMatchNoDocs = false; } originalIndices = OriginalIndices.readOriginalIndices(in); } @@ -210,7 +210,7 @@ protected final void innerWriteTo(StreamOutput out, boolean asKey) throws IOExce out.writeOptionalString(preference); } if (out.getVersion().onOrAfter(Version.V_8_0_0)) { - out.writeBoolean(matchNoDocsReturnNullResponse); + out.writeBoolean(canReturnNullResponseIfMatchNoDocs); } } @@ -291,12 +291,12 @@ public String preference() { * Defaults to false since the coordinator node needs at least one shard response to build the global * response. */ - public boolean isMatchNoDocsReturnNullResponse() { - return matchNoDocsReturnNullResponse; + public boolean canReturnNullResponseIfMatchNoDocs() { + return canReturnNullResponseIfMatchNoDocs; } - public void setMatchNoDocsReturnNullResponse(boolean value) { - this.matchNoDocsReturnNullResponse = value; + public void canReturnNullResponseIfMatchNoDocs(boolean value) { + this.canReturnNullResponseIfMatchNoDocs = value; } /** From 6016fa43c844868940c1cf4f5b8572ab69093d52 Mon Sep 17 00:00:00 2001 From: jimczi Date: Mon, 3 Feb 2020 11:49:40 +0100 Subject: [PATCH 06/37] adapt test --- .../test/java/org/elasticsearch/search/SearchServiceTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java b/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java index 75435f40f7dfb..c9dc231f0997b 100644 --- a/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java +++ b/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java @@ -872,7 +872,7 @@ public void onFailure(Exception exc) { { CountDownLatch latch = new CountDownLatch(1); - shardRequest.setMatchNoDocsReturnNullResponse(true); + shardRequest.canReturnNullResponseIfMatchNoDocs(true); service.executeQueryPhase(shardRequest, task, new ActionListener<>() { @Override public void onResponse(SearchPhaseResult result) { From a0581270a25c6a11182a5eae9ec2cdeec39e2a07 Mon Sep 17 00:00:00 2001 From: jimczi Date: Mon, 3 Feb 2020 11:52:09 +0100 Subject: [PATCH 07/37] fix test --- .../search/internal/ShardSearchRequestTests.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java b/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java index 2e2384f0f84fd..2d0aa2f591d8d 100644 --- a/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java +++ b/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java @@ -75,8 +75,8 @@ public void testSerialization() throws Exception { assertEquals(deserializedRequest.indexBoost(), shardSearchTransportRequest.indexBoost(), 0.0f); assertEquals(deserializedRequest.getClusterAlias(), shardSearchTransportRequest.getClusterAlias()); assertEquals(shardSearchTransportRequest.allowPartialSearchResults(), deserializedRequest.allowPartialSearchResults()); - assertEquals(deserializedRequest.isMatchNoDocsReturnNullResponse(), - shardSearchTransportRequest.isMatchNoDocsReturnNullResponse()); + assertEquals(deserializedRequest.canReturnNullResponseIfMatchNoDocs(), + shardSearchTransportRequest.canReturnNullResponseIfMatchNoDocs()); } private ShardSearchRequest createShardSearchRequest() throws IOException { @@ -93,7 +93,7 @@ private ShardSearchRequest createShardSearchRequest() throws IOException { ShardSearchRequest req = new ShardSearchRequest(new OriginalIndices(searchRequest), searchRequest, shardId, randomIntBetween(1, 100), filteringAliases, randomBoolean() ? 1.0f : randomFloat(), Math.abs(randomLong()), randomAlphaOfLengthBetween(3, 10), routings); - req.setMatchNoDocsReturnNullResponse(randomBoolean()); + req.canReturnNullResponseIfMatchNoDocs(randomBoolean()); return req; } From 8534ed25df43ced97d8ff983caaaec115f6d0ea6 Mon Sep 17 00:00:00 2001 From: jimczi Date: Mon, 3 Feb 2020 15:24:37 +0100 Subject: [PATCH 08/37] fix topNSize when size is reset to 0 --- .../action/search/SearchPhaseController.java | 22 ++++++++++++++++--- 1 file changed, 19 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java index 06bb29a51ce6c..59a5082ffe922 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java @@ -39,6 +39,7 @@ import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchPhaseResult; +import org.elasticsearch.search.SearchService; import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation.ReduceContext; @@ -507,6 +508,18 @@ private ReducedQueryPhase reducedQueryPhase(Collection= 2 if there is more than one expected result"); @@ -620,6 +634,7 @@ private QueryPhaseResultConsumer(SearchProgressListener progressListener, Search this.hasAggs = hasAggs; this.bufferSize = bufferSize; this.topDocsStats = new TopDocsStats(trackTotalHitsUpTo); + this.topNSize = topNSize; this.performFinalReduce = performFinalReduce; } @@ -643,7 +658,7 @@ private synchronized void consumeInternal(QuerySearchResult querySearchResult) { if (hasTopDocs) { TopDocs reducedTopDocs = mergeTopDocs(Arrays.asList(topDocsBuffer), // we have to merge here in the same way we collect on a shard - querySearchResult.from() + querySearchResult.size(), 0); + topNSize, 0); Arrays.fill(topDocsBuffer, null); topDocsBuffer[0] = reducedTopDocs; } @@ -718,9 +733,10 @@ ArraySearchPhaseResults newSearchPhaseResults(SearchProgressL if (isScrollRequest == false && (hasAggs || hasTopDocs)) { // no incremental reduce if scroll is used - we only hit a single shard or sometimes more... if (request.getBatchedReduceSize() < numShards) { + int topNSize = getTopDocsSize(request); // only use this if there are aggs and if there are more shards than we should reduce at once return new QueryPhaseResultConsumer(listener, this, numShards, request.getBatchedReduceSize(), hasTopDocs, hasAggs, - trackTotalHitsUpTo, request.isFinalReduce()); + trackTotalHitsUpTo, topNSize, request.isFinalReduce()); } } return new ArraySearchPhaseResults(numShards) { From 27cdf19dfe2c98cd9939cb15f1cab7b34a6e5254 Mon Sep 17 00:00:00 2001 From: jimczi Date: Mon, 3 Feb 2020 20:44:59 +0100 Subject: [PATCH 09/37] add more comments --- .../src/main/java/org/elasticsearch/search/SearchService.java | 1 + .../java/org/elasticsearch/index/engine/FrozenIndexTests.java | 4 ++-- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/SearchService.java b/server/src/main/java/org/elasticsearch/search/SearchService.java index 2d55e86c2fa3a..1750fe41e0d07 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchService.java +++ b/server/src/main/java/org/elasticsearch/search/SearchService.java @@ -1125,6 +1125,7 @@ private void rewriteShardRequest(ShardSearchRequest request, ActionListener { try { + // we can now acquire a searcher and rewrite the request with it SearchRewriteContext rewriteContext = acquireSearcherAndRewrite(request, shard); listener.onResponse(rewriteContext); } catch (Exception e) { diff --git a/x-pack/plugin/frozen-indices/src/test/java/org/elasticsearch/index/engine/FrozenIndexTests.java b/x-pack/plugin/frozen-indices/src/test/java/org/elasticsearch/index/engine/FrozenIndexTests.java index 3a976a2cd14b8..2bcaf0a926609 100644 --- a/x-pack/plugin/frozen-indices/src/test/java/org/elasticsearch/index/engine/FrozenIndexTests.java +++ b/x-pack/plugin/frozen-indices/src/test/java/org/elasticsearch/index/engine/FrozenIndexTests.java @@ -123,8 +123,8 @@ public void testSearchAndGetAPIsAreThrottled() throws InterruptedException, IOEx int numRefreshes = 0; for (int i = 0; i < numRequests; i++) { numRefreshes++; - // make sure that we don't share the frozen reader in - // concurrent requests + // make sure that we don't share the frozen reader in concurrent requests since we acquire the + // searcher and rewrite the request outside of the search-throttle thread pool CountDownLatch reqLatch = new CountDownLatch(1); switch (randomFrom(Arrays.asList(0, 1, 2))) { case 0: From 662972cf2de9b2ad36077b886c4930c8d2a08468 Mon Sep 17 00:00:00 2001 From: jimczi Date: Mon, 3 Feb 2020 21:36:16 +0100 Subject: [PATCH 10/37] more review --- .../search/AbstractSearchAsyncAction.java | 4 +-- .../index/engine/FrozenIndexTests.java | 32 ++++--------------- 2 files changed, 8 insertions(+), 28 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index bd1a3bdd3123c..d4d313c0afab1 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -76,7 +76,7 @@ abstract class AbstractSearchAsyncAction exten **/ private final BiFunction nodeIdToConnection; private final SearchTask task; - protected final SearchPhaseResults results; + private final SearchPhaseResults results; private final long clusterStateVersion; private final Map aliasFilter; private final Map concreteIndexBoosts; @@ -464,7 +464,7 @@ public final void onShardFailure(final int shardIndex, @Nullable SearchShardTarg * @param result the result returned form the shard * @param shardIt the shard iterator */ - protected void onShardResult(Result result, SearchShardIterator shardIt) { + private void onShardResult(Result result, SearchShardIterator shardIt) { assert result.getShardIndex() != -1 : "shard index is not set"; assert result.getSearchShardTarget() != null : "search shard target must not be null"; successfulOps.incrementAndGet(); diff --git a/x-pack/plugin/frozen-indices/src/test/java/org/elasticsearch/index/engine/FrozenIndexTests.java b/x-pack/plugin/frozen-indices/src/test/java/org/elasticsearch/index/engine/FrozenIndexTests.java index 2bcaf0a926609..eb89391fac177 100644 --- a/x-pack/plugin/frozen-indices/src/test/java/org/elasticsearch/index/engine/FrozenIndexTests.java +++ b/x-pack/plugin/frozen-indices/src/test/java/org/elasticsearch/index/engine/FrozenIndexTests.java @@ -6,7 +6,6 @@ package org.elasticsearch.index.engine; import org.elasticsearch.ResourceNotFoundException; -import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; @@ -49,7 +48,6 @@ import java.util.Arrays; import java.util.Collection; import java.util.EnumSet; -import java.util.concurrent.CountDownLatch; import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; @@ -119,55 +117,37 @@ public void testSearchAndGetAPIsAreThrottled() throws InterruptedException, IOEx } assertAcked(client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest("index")).actionGet()); int numRequests = randomIntBetween(20, 50); - CountDownLatch latch = new CountDownLatch(numRequests); int numRefreshes = 0; for (int i = 0; i < numRequests; i++) { numRefreshes++; // make sure that we don't share the frozen reader in concurrent requests since we acquire the // searcher and rewrite the request outside of the search-throttle thread pool - CountDownLatch reqLatch = new CountDownLatch(1); switch (randomFrom(Arrays.asList(0, 1, 2))) { case 0: client().prepareGet("index", "" + randomIntBetween(0, 9)) - .execute(ActionListener.wrap(() -> { - latch.countDown(); - reqLatch.countDown(); - })); - reqLatch.await(); + .get(); break; case 1: client().prepareSearch("index").setIndicesOptions(IndicesOptions.STRICT_EXPAND_OPEN_FORBID_CLOSED) .setSearchType(SearchType.QUERY_THEN_FETCH) - .execute(ActionListener.wrap(() -> { - latch.countDown(); - reqLatch.countDown(); - })); + .get(); // in total 4 refreshes 1x query & 1x fetch per shard (we have 2) numRefreshes += 3; - reqLatch.await(); break; case 2: client().prepareTermVectors("index", "" + randomIntBetween(0, 9)) - .execute(ActionListener.wrap(() -> { - latch.countDown(); - reqLatch.countDown(); - })); - reqLatch.await(); + .get(); break; case 3: - client().prepareExplain("index", "" + randomIntBetween(0, 9)).setQuery(new MatchAllQueryBuilder()) - .execute(ActionListener.wrap(() -> { - latch.countDown(); - reqLatch.countDown(); - })); - reqLatch.await(); + client().prepareExplain("index", "" + randomIntBetween(0, 9)) + .setQuery(new MatchAllQueryBuilder()) + .get(); break; default: assert false; } } - latch.await(); IndicesStatsResponse index = client().admin().indices().prepareStats("index").clear().setRefresh(true).get(); assertEquals(numRefreshes, index.getTotal().refresh.getTotal()); } From 76e90a231f3d4ac4475425e0326792f845b2b4ac Mon Sep 17 00:00:00 2001 From: jimczi Date: Mon, 3 Feb 2020 21:38:39 +0100 Subject: [PATCH 11/37] initial commit --- .../BulkByScrollParallelizationHelper.java | 2 +- .../search/140_pre_filter_search_shards.yml | 50 ++++++- .../action/search/SearchPhaseController.java | 24 +++- .../SearchQueryThenFetchAsyncAction.java | 60 ++++++++- .../search/builder/SearchSourceBuilder.java | 6 +- .../search/internal/ShardSearchRequest.java | 43 +++++- .../search/sort/FieldSortBuilder.java | 23 ++++ .../SearchQueryThenFetchAsyncActionTests.java | 124 ++++++++++++++++++ .../search/SearchServiceTests.java | 41 +++++- .../internal/ShardSearchRequestTests.java | 11 +- .../search/sort/FieldSortBuilderTests.java | 9 ++ 11 files changed, 366 insertions(+), 27 deletions(-) create mode 100644 server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/BulkByScrollParallelizationHelper.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/BulkByScrollParallelizationHelper.java index fbcb3d51efac3..6f1948ad70167 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/BulkByScrollParallelizationHelper.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/BulkByScrollParallelizationHelper.java @@ -203,7 +203,7 @@ static SearchRequest[] sliceIntoSubRequests(SearchRequest request, String field, if (request.source().slice() != null) { throw new IllegalStateException("Can't slice a request that already has a slice configuration"); } - slicedSource = request.source().copyWithNewSlice(sliceBuilder); + slicedSource = request.source().shallowCopy().slice(sliceBuilder); } SearchRequest searchRequest = new SearchRequest(request); searchRequest.source(slicedSource); 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 636f3d88e10cf..77903af705c52 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 @@ -154,10 +154,9 @@ setup: - match: { hits.total: 2 } - length: { aggregations.idx_terms.buckets: 2 } - # check that empty responses are correctly handled when rewriting to match_no_docs + # check that empty responses are correctly handled when rewriting to match_no_docs - do: search: - rest_total_hits_as_int: true # ensure that one shard can return empty response max_concurrent_shard_requests: 1 body: { "size" : 0, "query" : { "range" : { "created_at" : { "gte" : "2016-02-01", "lt": "2018-02-01"}}}, "aggs" : { "idx_terms" : { "terms" : { "field" : "_index" } } } } @@ -166,12 +165,11 @@ setup: - match: { _shards.successful: 3 } - match: { _shards.skipped : 0 } - match: { _shards.failed: 0 } - - match: { hits.total: 2 } + - match: { hits.total.value: 2 } - length: { aggregations.idx_terms.buckets: 2 } - do: search: - rest_total_hits_as_int: true # ensure that one shard can return empty response max_concurrent_shard_requests: 2 body: { "size" : 0, "query" : { "range" : { "created_at" : { "gte" : "2019-02-01"}}}, "aggs" : { "idx_terms" : { "terms" : { "field" : "_index" } } } } @@ -180,5 +178,47 @@ setup: - match: { _shards.successful: 3 } - match: { _shards.skipped : 0 } - match: { _shards.failed: 0 } - - match: { hits.total: 0 } + - match: { hits.total.value: 0 } - length: { aggregations.idx_terms.buckets: 0 } + + # check field sort is correct when skipping query phase + - do: + search: + # ensure that one shard can return empty response + max_concurrent_shard_requests: 1 + pre_filter_shard_size: 1 + body: + "size": 1 + "track_total_hits": 1 + "sort": [{ "timestamp": "desc"}] + + - match: { _shards.total: 3 } + - match: { _shards.successful: 3 } + - match: { _shards.skipped: 0 } + - match: { _shards.failed: 0 } + - match: { hits.total.value: 1 } + - match: { hits.total.relation: "gte" } + - length: { hits.hits: 1 } + - match: { hits.hits.0._id: "3" } + + # same with aggs + - do: + search: + # ensure that one shard can return empty response + max_concurrent_shard_requests: 1 + pre_filter_shard_size: 1 + body: + "size": 1 + "track_total_hits": 1 + "sort": [{ "timestamp": "desc"}] + "aggs" : { "idx_terms" : { "terms" : { "field" : "_index" } } } + + - match: { _shards.total: 3 } + - match: { _shards.successful: 3 } + - match: { _shards.skipped: 2 } + - match: { _shards.failed: 0 } + - match: { hits.total.value: 3 } + - length: { hits.hits: 1 } + - match: { hits.total.relation: "gte" } + - match: {hits.hits.0._id: "1" } + - length: { aggregations.idx_terms.buckets: 3 } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java index 06bb29a51ce6c..1bec1dd64f052 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java @@ -67,6 +67,8 @@ import java.util.function.IntFunction; import java.util.stream.Collectors; +import static org.elasticsearch.action.search.SearchQueryThenFetchAsyncAction.getTopDocsSize; + public final class SearchPhaseController { private static final ScoreDoc[] EMPTY_DOCS = new ScoreDoc[0]; @@ -587,6 +589,7 @@ static final class QueryPhaseResultConsumer extends ArraySearchPhaseResults= 2 if there is more than one expected result"); @@ -620,6 +623,7 @@ private QueryPhaseResultConsumer(SearchProgressListener progressListener, Search this.hasAggs = hasAggs; this.bufferSize = bufferSize; this.topDocsStats = new TopDocsStats(trackTotalHitsUpTo); + this.topNSize = topNSize; this.performFinalReduce = performFinalReduce; } @@ -643,7 +647,7 @@ private synchronized void consumeInternal(QuerySearchResult querySearchResult) { if (hasTopDocs) { TopDocs reducedTopDocs = mergeTopDocs(Arrays.asList(topDocsBuffer), // we have to merge here in the same way we collect on a shard - querySearchResult.from() + querySearchResult.size(), 0); + topNSize, 0); Arrays.fill(topDocsBuffer, null); topDocsBuffer[0] = reducedTopDocs; } @@ -668,6 +672,15 @@ private synchronized void consumeInternal(QuerySearchResult querySearchResult) { processedShards[querySearchResult.getShardIndex()] = querySearchResult.getSearchShardTarget(); } + public TopDocs getBufferTopDocs() { + if (hasTopDocs) { + synchronized (this) { + return topDocsBuffer[0]; + } + } + return null; + } + private synchronized List getRemainingAggs() { return hasAggs ? Arrays.asList(aggsBuffer).subList(0, index) : null; } @@ -718,9 +731,10 @@ ArraySearchPhaseResults newSearchPhaseResults(SearchProgressL if (isScrollRequest == false && (hasAggs || hasTopDocs)) { // no incremental reduce if scroll is used - we only hit a single shard or sometimes more... if (request.getBatchedReduceSize() < numShards) { + int topNSize = getTopDocsSize(request); // only use this if there are aggs and if there are more shards than we should reduce at once return new QueryPhaseResultConsumer(listener, this, numShards, request.getBatchedReduceSize(), hasTopDocs, hasAggs, - trackTotalHitsUpTo, request.isFinalReduce()); + trackTotalHitsUpTo, topNSize, request.isFinalReduce()); } } return new ArraySearchPhaseResults(numShards) { diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index d5060b728347d..86862ccead270 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -20,22 +20,33 @@ package org.elasticsearch.action.search; import org.apache.logging.log4j.Logger; +import org.apache.lucene.search.FieldDoc; +import org.apache.lucene.search.TopDocs; +import org.apache.lucene.search.TopFieldDocs; import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.routing.GroupShardsIterator; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.search.SearchPhaseResult; +import org.elasticsearch.search.SearchService; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.internal.AliasFilter; +import org.elasticsearch.search.internal.ShardSearchRequest; +import org.elasticsearch.search.sort.FieldSortBuilder; import org.elasticsearch.transport.Transport; import java.util.Map; import java.util.Set; import java.util.concurrent.Executor; import java.util.function.BiFunction; +import java.util.function.Supplier; + +import static org.elasticsearch.search.internal.SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO; final class SearchQueryThenFetchAsyncAction extends AbstractSearchAsyncAction { private final SearchPhaseController searchPhaseController; + private final Supplier topDocsSupplier; + private final int topDocsSize; private final SearchProgressListener progressListener; SearchQueryThenFetchAsyncAction(final Logger logger, final SearchTransportService searchTransportService, @@ -49,9 +60,10 @@ final class SearchQueryThenFetchAsyncAction extends AbstractSearchAsyncAction listener) { + ShardSearchRequest request = rewriteShardRequest(buildShardSearchRequest(shardIt)); getSearchTransport().sendExecuteQuery(getConnection(shardIt.getClusterAlias(), shard.currentNodeId()), - buildShardSearchRequest(shardIt), getTask(), listener); + request, getTask(), listener); } @Override @@ -72,4 +85,47 @@ protected void onShardGroupFailure(int shardIndex, Exception exc) { protected SearchPhase getNextPhase(final SearchPhaseResults results, final SearchPhaseContext context) { return new FetchSearchPhase(results, searchPhaseController, context); } + + ShardSearchRequest rewriteShardRequest(ShardSearchRequest request) { + TopDocs topDocs = topDocsSupplier.get(); + if (topDocs != null && topDocs instanceof TopFieldDocs) { + SearchSourceBuilder source = request.source(); + int trackTotalHits = source.trackTotalHitsUpTo() == null ? DEFAULT_TRACK_TOTAL_HITS_UP_TO : + source.trackTotalHitsUpTo(); + if (topDocs.totalHits.value >= trackTotalHits) { + SearchSourceBuilder newSource = source.shallowCopy(); + newSource.trackTotalHits(false); + if (topDocs.scoreDocs.length >= topDocsSize) { + FieldDoc bottomDoc = (FieldDoc) topDocs.scoreDocs[topDocs.scoreDocs.length-1]; + request.setRawBottomSortValues(bottomDoc.fields); + } + request.source(newSource); + } + } + return request; + } + + static Supplier getTopDocsSupplier(SearchRequest request, SearchPhaseResults searchPhaseResults) { + if (searchPhaseResults instanceof SearchPhaseController.QueryPhaseResultConsumer == false) { + return () -> null; + } + int size = getTopDocsSize(request); + FieldSortBuilder fieldSort = FieldSortBuilder.getPrimaryFieldSortOrNull(request.source()); + if (size == 0 + || fieldSort == null + || fieldSort.getNestedSort() != null + || (fieldSort.missing() != null && "_last".equals(fieldSort.missing()) == false)) { + return () -> null; + } + return ((SearchPhaseController.QueryPhaseResultConsumer) searchPhaseResults)::getBufferTopDocs; + } + + static int getTopDocsSize(SearchRequest request) { + if (request.source() == null) { + return SearchService.DEFAULT_SIZE; + } + SearchSourceBuilder source = request.source(); + return (source.size() == -1 ? SearchService.DEFAULT_SIZE : source.size()) + + (source.from() == -1 ? SearchService.DEFAULT_FROM : source.from()); + } } diff --git a/server/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java b/server/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java index 857bc235a40ff..e708ee804352b 100644 --- a/server/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java @@ -959,13 +959,13 @@ public SearchSourceBuilder rewrite(QueryRewriteContext context) throws IOExcepti /** * Create a shallow copy of this builder with a new slice configuration. */ - public SearchSourceBuilder copyWithNewSlice(SliceBuilder slice) { - return shallowCopy(queryBuilder, postQueryBuilder, aggregations, slice, sorts, rescoreBuilders, highlightBuilder); + public SearchSourceBuilder shallowCopy() { + return shallowCopy(queryBuilder, postQueryBuilder, aggregations, sliceBuilder, sorts, rescoreBuilders, highlightBuilder); } /** * Create a shallow copy of this source replaced {@link #queryBuilder}, {@link #postQueryBuilder}, and {@link #sliceBuilder}. Used by - * {@link #rewrite(QueryRewriteContext)} and {@link #copyWithNewSlice(SliceBuilder)}. + * {@link #rewrite(QueryRewriteContext)}}. */ @SuppressWarnings("rawtypes") private SearchSourceBuilder shallowCopy(QueryBuilder queryBuilder, QueryBuilder postQueryBuilder, diff --git a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java index f46be3755cee5..7a7ebad16c395 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java +++ b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java @@ -37,17 +37,21 @@ import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.index.Index; import org.elasticsearch.index.query.BoolQueryBuilder; +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.Rewriteable; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.AliasFilterParsingException; import org.elasticsearch.indices.InvalidAliasNameException; import org.elasticsearch.search.Scroll; import org.elasticsearch.search.builder.SearchSourceBuilder; -import org.elasticsearch.search.query.QuerySearchResult; +import org.elasticsearch.search.sort.FieldSortBuilder; +import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskId; import org.elasticsearch.transport.TransportRequest; @@ -57,6 +61,9 @@ import java.util.Map; import java.util.function.Function; +import static org.elasticsearch.index.mapper.DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER; +import static org.elasticsearch.search.internal.SearchContext.TRACK_TOTAL_HITS_DISABLED; + /** * Shard level request that represents a search. * It provides all the methods that the {@link SearchContext} needs. @@ -77,6 +84,7 @@ public class ShardSearchRequest extends TransportRequest implements IndicesReque private final OriginalIndices originalIndices; private boolean canReturnNullResponseIfMatchNoDocs; + private Object[] rawBottomSortValues; //these are the only mutable fields, as they are subject to rewriting private AliasFilter aliasFilter; @@ -95,7 +103,7 @@ public ShardSearchRequest(OriginalIndices originalIndices, shardId, numberOfShards, searchRequest.searchType(), - searchRequest.source(), + searchRequest.source() != null ? searchRequest.source().shallowCopy() : null, searchRequest.requestCache(), aliasFilter, indexBoost, @@ -172,8 +180,10 @@ public ShardSearchRequest(StreamInput in) throws IOException { preference = in.readOptionalString(); if (in.getVersion().onOrAfter(Version.V_8_0_0)) { canReturnNullResponseIfMatchNoDocs = in.readBoolean(); + rawBottomSortValues = in.readOptionalArray(Lucene::readSortValue, Object[]::new); } else { canReturnNullResponseIfMatchNoDocs = false; + rawBottomSortValues = null; } originalIndices = OriginalIndices.readOriginalIndices(in); } @@ -211,6 +221,7 @@ protected final void innerWriteTo(StreamOutput out, boolean asKey) throws IOExce } if (out.getVersion().onOrAfter(Version.V_8_0_0)) { out.writeBoolean(canReturnNullResponseIfMatchNoDocs); + out.writeOptionalArray(Lucene::writeSortValue, rawBottomSortValues); } } @@ -286,11 +297,14 @@ public String preference() { return preference; } - /** - * Returns true if the caller can handle null response {@link QuerySearchResult#nullInstance()}. - * Defaults to false since the coordinator node needs at least one shard response to build the global - * response. - */ + public void setRawBottomSortValues(Object[] values) { + this.rawBottomSortValues = values; + } + + public Object[] getRawBottomSortValues() { + return rawBottomSortValues; + } + public boolean canReturnNullResponseIfMatchNoDocs() { return canReturnNullResponseIfMatchNoDocs; } @@ -343,6 +357,21 @@ static class RequestRewritable implements Rewriteable { public Rewriteable rewrite(QueryRewriteContext ctx) throws IOException { SearchSourceBuilder newSource = request.source() == null ? null : Rewriteable.rewrite(request.source(), ctx); AliasFilter newAliasFilter = Rewriteable.rewrite(request.getAliasFilter(), ctx); + + QueryShardContext shardContext = ctx.convertToShardContext(); + if (shardContext != null + && FieldSortBuilder.isBottomSortDisjoint(shardContext, newSource, request.getRawBottomSortValues())) { + newSource = newSource.shallowCopy(); + if (newSource.trackTotalHitsUpTo() == TRACK_TOTAL_HITS_DISABLED + && newSource.aggregations() == null) { + newSource.query(new MatchNoneQueryBuilder()); + } else { + newSource.size(0); + } + request.source(newSource); + request.setRawBottomSortValues(null); + } + if (newSource == request.source() && newAliasFilter == request.getAliasFilter()) { return this; } else { diff --git a/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java b/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java index 7a7cf1437888d..ca195e571495f 100644 --- a/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java @@ -60,6 +60,7 @@ import java.util.Objects; import java.util.function.Function; +import static org.elasticsearch.index.mapper.DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER; import static org.elasticsearch.index.mapper.DateFieldMapper.Resolution.MILLISECONDS; import static org.elasticsearch.index.mapper.DateFieldMapper.Resolution.NANOSECONDS; import static org.elasticsearch.index.search.NestedHelper.parentObject; @@ -394,6 +395,28 @@ public static FieldSortBuilder getPrimaryFieldSortOrNull(SearchSourceBuilder sou return source.sorts().get(0) instanceof FieldSortBuilder ? (FieldSortBuilder) source.sorts().get(0) : null; } + public static boolean isBottomSortDisjoint(QueryShardContext context, + SearchSourceBuilder source, + Object[] rawBottomSortValues) throws IOException { + if (rawBottomSortValues == null || rawBottomSortValues.length == 0) { + return false; + } + FieldSortBuilder fieldSort = FieldSortBuilder.getPrimaryFieldSortOrNull(source); + if (fieldSort == null) { + return false; + } + MappedFieldType fieldType = context.fieldMapper(fieldSort.getFieldName()); + if (fieldType == null) { + // unmapped + return true; + } + Object minValue = fieldSort.order() == SortOrder.DESC ? rawBottomSortValues[0] : null; + Object maxValue = fieldSort.order() == SortOrder.DESC ? null : rawBottomSortValues[0]; + MappedFieldType.Relation relation = fieldType.isFieldWithinQuery(context.searcher().getIndexReader(), minValue, maxValue, + true, true, null, DEFAULT_DATE_TIME_FORMATTER.toDateMathParser(), context); + return relation == MappedFieldType.Relation.DISJOINT; + } + /** * Return a {@link Function} that converts a serialized point into a {@link Number} according to the provided * {@link SortField}. This is needed for {@link SortField} that converts values from one type to another using diff --git a/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java b/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java new file mode 100644 index 0000000000000..8311730e3822e --- /dev/null +++ b/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java @@ -0,0 +1,124 @@ +package org.elasticsearch.action.search; + +import org.apache.lucene.search.FieldDoc; +import org.apache.lucene.search.SortField; +import org.apache.lucene.search.TopFieldDocs; +import org.apache.lucene.search.TotalHits; +import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.OriginalIndices; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.routing.GroupShardsIterator; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.lucene.search.TopDocsAndMaxScore; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.concurrent.EsExecutors; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.SearchPhaseResult; +import org.elasticsearch.search.SearchShardTarget; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.builder.SearchSourceBuilder; +import org.elasticsearch.search.internal.AliasFilter; +import org.elasticsearch.search.internal.ShardSearchRequest; +import org.elasticsearch.search.query.QuerySearchResult; +import org.elasticsearch.search.sort.SortBuilders; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.transport.Transport; + +import java.util.Collections; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.instanceOf; + +public class SearchQueryThenFetchAsyncActionTests extends ESTestCase { + public void testSearchBefore() 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)); + + int numShards = randomIntBetween(10, 20); + int numConcurrent = randomIntBetween(1, 4); + AtomicInteger numWithTopDocs = new AtomicInteger(); + AtomicInteger successfulOps = new AtomicInteger(); + SearchTransportService searchTransportService = new SearchTransportService(null, null) { + @Override + public void sendExecuteQuery(Transport.Connection connection, ShardSearchRequest request, SearchTask task, SearchActionListener listener) { + int shardId = request.shardId().id(); + if (request.getRawBottomSortValues() != null) { + assertNotEquals(shardId, (int) request.getRawBottomSortValues()[0]); + numWithTopDocs.incrementAndGet(); + } + QuerySearchResult queryResult = new QuerySearchResult(123, + new SearchShardTarget("node1", new ShardId("test", "na", shardId), null, OriginalIndices.NONE)); + SortField sortField = new SortField("timestamp", SortField.Type.LONG); + queryResult.topDocs(new TopDocsAndMaxScore(new TopFieldDocs( + new TotalHits(1, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO), + new FieldDoc[] { + new FieldDoc(randomInt(1000), Float.NaN, new Object[] { request.shardId().id() }) + }, new SortField[] { sortField }), Float.NaN), + new DocValueFormat[] { DocValueFormat.RAW }); + queryResult.from(0); + queryResult.size(1); + successfulOps.incrementAndGet(); + new Thread(() -> listener.onResponse(queryResult)).start(); + } + }; + CountDownLatch latch = new CountDownLatch(1); + GroupShardsIterator shardsIter = SearchAsyncActionTests.getShardsIter("idx", + new OriginalIndices(new String[]{"idx"}, SearchRequest.DEFAULT_INDICES_OPTIONS), + numShards, randomBoolean(), primaryNode, replicaNode); + final SearchRequest searchRequest = new SearchRequest(); + searchRequest.setMaxConcurrentShardRequests(numConcurrent); + searchRequest.setBatchedReduceSize(2); + searchRequest.source(new SearchSourceBuilder() + .size(1) + .trackTotalHitsUpTo(1) + .sort(SortBuilders.fieldSort("timestamp"))); + searchRequest.allowPartialSearchResults(false); + SearchPhaseController controller = new SearchPhaseController((b) -> new InternalAggregation.ReduceContext(BigArrays.NON_RECYCLING_INSTANCE, null, b)); + SearchTask task = new SearchTask(0, "n/a", "n/a", "test", null, Collections.emptyMap()); + SearchQueryThenFetchAsyncAction action = new SearchQueryThenFetchAsyncAction(logger, + searchTransportService, (clusterAlias, node) -> lookup.get(node), Collections.singletonMap("_na_", new AliasFilter(null, Strings.EMPTY_ARRAY)), + Collections.emptyMap(), Collections.emptyMap(), controller, EsExecutors.newDirectExecutorService(), searchRequest, + new ActionListener<>() { + @Override + public void onResponse(SearchResponse response) { + latch.countDown(); + } + + @Override + public void onFailure(Exception e) { + try { + throw new AssertionError(e); + } finally { + latch.countDown(); + } + } + }, shardsIter, timeProvider, 0, task, + SearchResponse.Clusters.EMPTY); + action.start(); + latch.await(); + assertThat(successfulOps.get(), equalTo(numShards)); + assertThat(numWithTopDocs.get(), greaterThanOrEqualTo(1)); + SearchPhaseController.ReducedQueryPhase phase = action.results.reduce(); + assertThat(phase.numReducePhases, greaterThanOrEqualTo(1)); + assertThat(phase.totalHits.value, equalTo(1L)); + assertThat(phase.totalHits.relation, equalTo(TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO)); + assertThat(phase.sortedTopDocs.scoreDocs.length, equalTo(1)); + assertThat(phase.sortedTopDocs.scoreDocs[0], instanceOf(FieldDoc.class)); + assertThat(((FieldDoc) phase.sortedTopDocs.scoreDocs[0]).fields.length, equalTo(1)); + assertThat(((FieldDoc) phase.sortedTopDocs.scoreDocs[0]).fields[0], equalTo(0)); + + } +} diff --git a/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java b/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java index 75435f40f7dfb..9c0b05b997933 100644 --- a/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java +++ b/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java @@ -19,11 +19,23 @@ package org.elasticsearch.search; import com.carrotsearch.hppc.IntArrayList; +import org.apache.lucene.codecs.CompoundFormat; +import org.apache.lucene.codecs.FilterCodec; +import org.apache.lucene.codecs.PostingsFormat; +import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat; +import org.apache.lucene.codecs.lucene70.Lucene70Codec; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.TextField; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.FilterDirectoryReader; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.LeafReader; import org.apache.lucene.search.Query; import org.apache.lucene.store.AlreadyClosedException; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.RAMDirectory; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.OriginalIndices; @@ -872,7 +884,7 @@ public void onFailure(Exception exc) { { CountDownLatch latch = new CountDownLatch(1); - shardRequest.setMatchNoDocsReturnNullResponse(true); + shardRequest.canReturnNullResponseIfMatchNoDocs(true); service.executeQueryPhase(shardRequest, task, new ActionListener<>() { @Override public void onResponse(SearchPhaseResult result) { @@ -898,4 +910,31 @@ public void onFailure(Exception e) { latch.await(); } } + + public void test() throws IOException { + IndexWriterConfig config = new IndexWriterConfig(); + Lucene70Codec codec = new Lucene70Codec(); + Lucene50PostingsFormat pf = new Lucene50PostingsFormat(); + config.setCodec(new FilterCodec("lucene7", codec) { + @Override + public CompoundFormat compoundFormat() { + return super.compoundFormat(); + } + + @Override + public PostingsFormat postingsFormat() { + return pf; + } + }); + Directory dir = new RAMDirectory(); + IndexWriter writer = new IndexWriter(dir, config); + for (int i = 0; i < 10; i++) { + Document doc = new Document(); + doc.add(new TextField("text", "hello", Field.Store.YES)); + writer.addDocument(doc); + } + writer.commit(); + writer.close(); + dir.close(); + } } diff --git a/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java b/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java index 2e2384f0f84fd..ca0eeb2bb46c2 100644 --- a/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java +++ b/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java @@ -75,8 +75,8 @@ public void testSerialization() throws Exception { assertEquals(deserializedRequest.indexBoost(), shardSearchTransportRequest.indexBoost(), 0.0f); assertEquals(deserializedRequest.getClusterAlias(), shardSearchTransportRequest.getClusterAlias()); assertEquals(shardSearchTransportRequest.allowPartialSearchResults(), deserializedRequest.allowPartialSearchResults()); - assertEquals(deserializedRequest.isMatchNoDocsReturnNullResponse(), - shardSearchTransportRequest.isMatchNoDocsReturnNullResponse()); + assertEquals(deserializedRequest.canReturnNullResponseIfMatchNoDocs(), + shardSearchTransportRequest.canReturnNullResponseIfMatchNoDocs()); } private ShardSearchRequest createShardSearchRequest() throws IOException { @@ -93,7 +93,12 @@ private ShardSearchRequest createShardSearchRequest() throws IOException { ShardSearchRequest req = new ShardSearchRequest(new OriginalIndices(searchRequest), searchRequest, shardId, randomIntBetween(1, 100), filteringAliases, randomBoolean() ? 1.0f : randomFloat(), Math.abs(randomLong()), randomAlphaOfLengthBetween(3, 10), routings); - req.setMatchNoDocsReturnNullResponse(randomBoolean()); + req.canReturnNullResponseIfMatchNoDocs(randomBoolean()); + if (randomBoolean()) { + req.setRawBottomSortValues( + randomArray(randomIntBetween(1, 3), Object[]::new, + () -> randomBoolean() ? randomUnicodeOfLength(10) : randomLong())); + } return req; } diff --git a/server/src/test/java/org/elasticsearch/search/sort/FieldSortBuilderTests.java b/server/src/test/java/org/elasticsearch/search/sort/FieldSortBuilderTests.java index b4482e5545503..b6605e8f9e10c 100644 --- a/server/src/test/java/org/elasticsearch/search/sort/FieldSortBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/search/sort/FieldSortBuilderTests.java @@ -69,6 +69,7 @@ import static org.elasticsearch.search.sort.FieldSortBuilder.getMinMaxOrNull; import static org.elasticsearch.search.sort.FieldSortBuilder.getPrimaryFieldSortOrNull; +import static org.elasticsearch.search.sort.FieldSortBuilder.isBottomSortDisjoint; import static org.elasticsearch.search.sort.NestedSortBuilderTests.createRandomNestedSort; import static org.hamcrest.Matchers.instanceOf; @@ -565,6 +566,14 @@ public void testGetMaxKeywordValue() throws IOException { } } + public void testIsBottomSortDisjoint() throws Exception { + QueryShardContext context = createMockShardContext(); + assertFalse(isBottomSortDisjoint(context, null, null)); + assertFalse(isBottomSortDisjoint(context, new SearchSourceBuilder(), null)); + assertFalse(isBottomSortDisjoint(context, + new SearchSourceBuilder().sort(SortBuilders.fieldSort("timestamp")), null)); + } + @Override protected FieldSortBuilder fromXContent(XContentParser parser, String fieldName) throws IOException { return FieldSortBuilder.fromXContent(parser, fieldName); From ac0451ca0a9f124e6567128a24450025050725a2 Mon Sep 17 00:00:00 2001 From: jimczi Date: Mon, 3 Feb 2020 23:25:00 +0100 Subject: [PATCH 12/37] iter --- .../search/140_pre_filter_search_shards.yml | 14 +++--- .../action/search/SearchPhaseController.java | 2 +- .../search/sort/FieldSortBuilderTests.java | 48 +++++++++++++++++-- 3 files changed, 51 insertions(+), 13 deletions(-) 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 77903af705c52..7e3b105bea700 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 @@ -190,7 +190,7 @@ setup: body: "size": 1 "track_total_hits": 1 - "sort": [{ "timestamp": "desc"}] + "sort": [{ "created_at": "desc"}] - match: { _shards.total: 3 } - match: { _shards.successful: 3 } @@ -199,7 +199,7 @@ setup: - match: { hits.total.value: 1 } - match: { hits.total.relation: "gte" } - length: { hits.hits: 1 } - - match: { hits.hits.0._id: "3" } + - match: { hits.hits.0._id: "3" } # same with aggs - do: @@ -210,15 +210,15 @@ setup: body: "size": 1 "track_total_hits": 1 - "sort": [{ "timestamp": "desc"}] + "sort": [{ "created_at": "desc"}] "aggs" : { "idx_terms" : { "terms" : { "field" : "_index" } } } - match: { _shards.total: 3 } - match: { _shards.successful: 3 } - - match: { _shards.skipped: 2 } + - match: { _shards.skipped: 0 } - match: { _shards.failed: 0 } - - match: { hits.total.value: 3 } - - length: { hits.hits: 1 } + - match: { hits.total.value: 1 } - match: { hits.total.relation: "gte" } - - match: {hits.hits.0._id: "1" } + - length: { hits.hits: 1 } + - match: {hits.hits.0._id: "3" } - length: { aggregations.idx_terms.buckets: 3 } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java index 1bec1dd64f052..bbf607718d9a3 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java @@ -588,8 +588,8 @@ static final class QueryPhaseResultConsumer extends ArraySearchPhaseResults Date: Tue, 4 Feb 2020 09:14:52 +0100 Subject: [PATCH 13/37] iter --- .../SearchQueryThenFetchAsyncAction.java | 5 +- .../search/internal/ShardSearchRequest.java | 6 +- .../search/sort/FieldSortBuilder.java | 57 ++++++++++++------- .../SearchQueryThenFetchAsyncActionTests.java | 11 ++-- .../search/sort/FieldSortBuilderTests.java | 49 ++++++++-------- 5 files changed, 70 insertions(+), 58 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 86862ccead270..1d4ccc38dc560 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -105,7 +105,7 @@ ShardSearchRequest rewriteShardRequest(ShardSearchRequest request) { return request; } - static Supplier getTopDocsSupplier(SearchRequest request, SearchPhaseResults searchPhaseResults) { + private Supplier getTopDocsSupplier(SearchRequest request, SearchPhaseResults searchPhaseResults) { if (searchPhaseResults instanceof SearchPhaseController.QueryPhaseResultConsumer == false) { return () -> null; } @@ -113,8 +113,7 @@ static Supplier getTopDocsSupplier(SearchRequest request, SearchPhaseRe FieldSortBuilder fieldSort = FieldSortBuilder.getPrimaryFieldSortOrNull(request.source()); if (size == 0 || fieldSort == null - || fieldSort.getNestedSort() != null - || (fieldSort.missing() != null && "_last".equals(fieldSort.missing()) == false)) { + || fieldSort.canRewriteToMatchNone() == false) { return () -> null; } return ((SearchPhaseController.QueryPhaseResultConsumer) searchPhaseResults)::getBufferTopDocs; diff --git a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java index 7a7ebad16c395..9842413e9b166 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java +++ b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java @@ -51,7 +51,6 @@ import org.elasticsearch.search.Scroll; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.sort.FieldSortBuilder; -import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskId; import org.elasticsearch.transport.TransportRequest; @@ -61,7 +60,6 @@ import java.util.Map; import java.util.function.Function; -import static org.elasticsearch.index.mapper.DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER; import static org.elasticsearch.search.internal.SearchContext.TRACK_TOTAL_HITS_DISABLED; /** @@ -359,8 +357,10 @@ public Rewriteable rewrite(QueryRewriteContext ctx) throws IOException { AliasFilter newAliasFilter = Rewriteable.rewrite(request.getAliasFilter(), ctx); QueryShardContext shardContext = ctx.convertToShardContext(); + FieldSortBuilder primarySort = FieldSortBuilder.getPrimaryFieldSortOrNull(newSource); if (shardContext != null - && FieldSortBuilder.isBottomSortDisjoint(shardContext, newSource, request.getRawBottomSortValues())) { + && primarySort != null + && primarySort.isBottomSortWithinShard(shardContext, request.getRawBottomSortValues()) == false) { newSource = newSource.shallowCopy(); if (newSource.trackTotalHitsUpTo() == TRACK_TOTAL_HITS_DISABLED && newSource.aggregations() == null) { diff --git a/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java b/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java index ca195e571495f..a4757b7bb587b 100644 --- a/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java @@ -376,6 +376,41 @@ public SortFieldAndFormat build(QueryShardContext context) throws IOException { return new SortFieldAndFormat(field, fieldType.docValueFormat(null, null)); } + public boolean canRewriteToMatchNone() { + return nestedSort == null && (missing == null || "_last".equals(missing)); + } + + /** + * Return true if the + */ + public boolean isBottomSortWithinShard(QueryShardContext context, + Object[] rawBottomSortValues) { + if (rawBottomSortValues == null || rawBottomSortValues.length == 0) { + return true; + } + + if (canRewriteToMatchNone() == false) { + return true; + } + MappedFieldType fieldType = context.fieldMapper(fieldName); + if (fieldType == null) { + // unmapped + return false; + } + if (fieldType.indexOptions() == IndexOptions.NONE) { + return true; + } + Object minValue = order() == SortOrder.DESC ? rawBottomSortValues[0] : null; + Object maxValue = order() == SortOrder.DESC ? null : rawBottomSortValues[0]; + try { + MappedFieldType.Relation relation = fieldType.isFieldWithinQuery(context.searcher().getIndexReader(), minValue, maxValue, + true, true, null, DEFAULT_DATE_TIME_FORMATTER.toDateMathParser(), context); + return relation != MappedFieldType.Relation.DISJOINT; + } catch (Exception exc) { + return true; + } + } + /** * Return true if the primary sort in the provided source * is an instance of {@link FieldSortBuilder}. @@ -395,28 +430,6 @@ public static FieldSortBuilder getPrimaryFieldSortOrNull(SearchSourceBuilder sou return source.sorts().get(0) instanceof FieldSortBuilder ? (FieldSortBuilder) source.sorts().get(0) : null; } - public static boolean isBottomSortDisjoint(QueryShardContext context, - SearchSourceBuilder source, - Object[] rawBottomSortValues) throws IOException { - if (rawBottomSortValues == null || rawBottomSortValues.length == 0) { - return false; - } - FieldSortBuilder fieldSort = FieldSortBuilder.getPrimaryFieldSortOrNull(source); - if (fieldSort == null) { - return false; - } - MappedFieldType fieldType = context.fieldMapper(fieldSort.getFieldName()); - if (fieldType == null) { - // unmapped - return true; - } - Object minValue = fieldSort.order() == SortOrder.DESC ? rawBottomSortValues[0] : null; - Object maxValue = fieldSort.order() == SortOrder.DESC ? null : rawBottomSortValues[0]; - MappedFieldType.Relation relation = fieldType.isFieldWithinQuery(context.searcher().getIndexReader(), minValue, maxValue, - true, true, null, DEFAULT_DATE_TIME_FORMATTER.toDateMathParser(), context); - return relation == MappedFieldType.Relation.DISJOINT; - } - /** * Return a {@link Function} that converts a serialized point into a {@link Number} according to the provided * {@link SortField}. This is needed for {@link SortField} that converts values from one type to another using diff --git a/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java b/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java index 8311730e3822e..b91ee4a1880a7 100644 --- a/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java @@ -37,7 +37,7 @@ import static org.hamcrest.Matchers.instanceOf; public class SearchQueryThenFetchAsyncActionTests extends ESTestCase { - public void testSearchBefore() throws InterruptedException { + public void testBottomFieldSort() throws InterruptedException { final TransportSearchAction.SearchTimeProvider timeProvider = new TransportSearchAction.SearchTimeProvider(0, System.nanoTime(), System::nanoTime); @@ -53,7 +53,8 @@ public void testSearchBefore() throws InterruptedException { AtomicInteger successfulOps = new AtomicInteger(); SearchTransportService searchTransportService = new SearchTransportService(null, null) { @Override - public void sendExecuteQuery(Transport.Connection connection, ShardSearchRequest request, SearchTask task, SearchActionListener listener) { + public void sendExecuteQuery(Transport.Connection connection, ShardSearchRequest request, + SearchTask task, SearchActionListener listener) { int shardId = request.shardId().id(); if (request.getRawBottomSortValues() != null) { assertNotEquals(shardId, (int) request.getRawBottomSortValues()[0]); @@ -86,10 +87,12 @@ public void sendExecuteQuery(Transport.Connection connection, ShardSearchRequest .trackTotalHitsUpTo(1) .sort(SortBuilders.fieldSort("timestamp"))); searchRequest.allowPartialSearchResults(false); - SearchPhaseController controller = new SearchPhaseController((b) -> new InternalAggregation.ReduceContext(BigArrays.NON_RECYCLING_INSTANCE, null, b)); + SearchPhaseController controller = new SearchPhaseController((b) -> + new InternalAggregation.ReduceContext(BigArrays.NON_RECYCLING_INSTANCE, null, b)); SearchTask task = new SearchTask(0, "n/a", "n/a", "test", null, Collections.emptyMap()); SearchQueryThenFetchAsyncAction action = new SearchQueryThenFetchAsyncAction(logger, - searchTransportService, (clusterAlias, node) -> lookup.get(node), Collections.singletonMap("_na_", new AliasFilter(null, Strings.EMPTY_ARRAY)), + searchTransportService, (clusterAlias, node) -> lookup.get(node), + Collections.singletonMap("_na_", new AliasFilter(null, Strings.EMPTY_ARRAY)), Collections.emptyMap(), Collections.emptyMap(), controller, EsExecutors.newDirectExecutorService(), searchRequest, new ActionListener<>() { @Override diff --git a/server/src/test/java/org/elasticsearch/search/sort/FieldSortBuilderTests.java b/server/src/test/java/org/elasticsearch/search/sort/FieldSortBuilderTests.java index c3d351575d26d..f41543c9c542b 100644 --- a/server/src/test/java/org/elasticsearch/search/sort/FieldSortBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/search/sort/FieldSortBuilderTests.java @@ -71,7 +71,6 @@ import static org.elasticsearch.search.sort.FieldSortBuilder.getMinMaxOrNull; import static org.elasticsearch.search.sort.FieldSortBuilder.getPrimaryFieldSortOrNull; -import static org.elasticsearch.search.sort.FieldSortBuilder.isBottomSortDisjoint; import static org.elasticsearch.search.sort.NestedSortBuilderTests.createRandomNestedSort; import static org.hamcrest.Matchers.instanceOf; @@ -570,15 +569,20 @@ public void testGetMaxKeywordValue() throws IOException { } } - public void testIsBottomSortDisjoint() throws Exception { + public void testIsBottomSortWithinShard() throws Exception { try (Directory dir = newDirectory()) { - int numDocs = randomIntBetween(10, 30); - long maxValue = Long.MIN_VALUE; - long minValue = Long.MAX_VALUE; + int numDocs = randomIntBetween(5, 10); + long maxValue = -1; + long minValue = Integer.MAX_VALUE; try (RandomIndexWriter writer = new RandomIndexWriter(random(), dir, new KeywordAnalyzer())) { + FieldSortBuilder fieldSort = SortBuilders.fieldSort("custom-date"); + try (DirectoryReader reader = writer.getReader()) { + QueryShardContext context = createMockShardContext(new IndexSearcher(reader)); + assertFalse(fieldSort.isBottomSortWithinShard(context, new Object[] { 0L })); + } for (int i = 0; i < numDocs; i++) { Document doc = new Document(); - long value = randomLongBetween(1, Long.MAX_VALUE-1); + long value = randomLongBetween(1, Integer.MAX_VALUE); doc.add(new LongPoint("custom-date", value)); doc.add(new SortedNumericDocValuesField("custom-date", value)); writer.addDocument(doc); @@ -587,26 +591,19 @@ public void testIsBottomSortDisjoint() throws Exception { } try (DirectoryReader reader = writer.getReader()) { QueryShardContext context = createMockShardContext(new IndexSearcher(reader)); - assertFalse(isBottomSortDisjoint(context, null, null)); - assertFalse(isBottomSortDisjoint(context, new SearchSourceBuilder(), null)); - assertFalse(isBottomSortDisjoint(context, - new SearchSourceBuilder().sort(SortBuilders.fieldSort("custom-date")), null)); - assertFalse(isBottomSortDisjoint(context, - new SearchSourceBuilder().sort(SortBuilders.fieldSort("custom-date")), new Object[]{minValue})); - assertTrue(isBottomSortDisjoint(context, - new SearchSourceBuilder().sort(SortBuilders.fieldSort("custom-date")), new Object[]{minValue-1})); - assertTrue(isBottomSortDisjoint(context, - new SearchSourceBuilder() - .sort(SortBuilders.fieldSort("custom-date").order(SortOrder.DESC)), new Object[]{maxValue+1})); - assertFalse(isBottomSortDisjoint(context, - new SearchSourceBuilder() - .sort(SortBuilders.fieldSort("custom-date").order(SortOrder.DESC)), new Object[]{maxValue})); - assertFalse(isBottomSortDisjoint(context, - new SearchSourceBuilder() - .sort(SortBuilders.fieldSort("custom-date").order(SortOrder.DESC)), new Object[]{minValue})); - assertFalse(isBottomSortDisjoint(context, - new SearchSourceBuilder() - .sort(SortBuilders.fieldSort("custom-date")), new Object[]{maxValue+1})); + assertTrue(fieldSort.isBottomSortWithinShard(context, null)); + assertTrue(fieldSort.isBottomSortWithinShard(context, new Object[] { minValue })); + assertFalse(fieldSort.isBottomSortWithinShard(context, new Object[] { minValue-1 })); + assertTrue(fieldSort.isBottomSortWithinShard(context, new Object[] { minValue+1 })); + fieldSort.order(SortOrder.DESC); + assertFalse(fieldSort.isBottomSortWithinShard(context, new Object[] { maxValue+1 })); + assertTrue(fieldSort.isBottomSortWithinShard(context, new Object[] { maxValue })); + assertTrue(fieldSort.isBottomSortWithinShard(context, new Object[] { minValue })); + fieldSort.setNestedSort(new NestedSortBuilder("empty")); + assertTrue(fieldSort.isBottomSortWithinShard(context, new Object[] { minValue-1 })); + fieldSort.setNestedSort(null); + fieldSort.missing("100"); + assertTrue(fieldSort.isBottomSortWithinShard(context, new Object[] { maxValue+1 })); } } } From 30b3bcb19c8f23baf30b3b5db2da261612f8fe78 Mon Sep 17 00:00:00 2001 From: jimczi Date: Tue, 4 Feb 2020 10:40:31 +0100 Subject: [PATCH 14/37] iter2 --- .../action/search/SearchPhaseController.java | 2 - .../SearchQueryThenFetchAsyncAction.java | 18 ++----- .../search/internal/ShardSearchRequest.java | 8 +++ .../search/sort/FieldSortBuilder.java | 5 +- .../SearchQueryThenFetchAsyncActionTests.java | 51 ++++++++++++------- 5 files changed, 49 insertions(+), 35 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java index c608659cba57a..e3aeecd4f5be9 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java @@ -68,8 +68,6 @@ import java.util.function.IntFunction; import java.util.stream.Collectors; -import static org.elasticsearch.action.search.SearchQueryThenFetchAsyncAction.getTopDocsSize; - public final class SearchPhaseController { private static final ScoreDoc[] EMPTY_DOCS = new ScoreDoc[0]; diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 1d4ccc38dc560..fa901ea8c1aea 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -27,7 +27,6 @@ import org.elasticsearch.cluster.routing.GroupShardsIterator; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.search.SearchPhaseResult; -import org.elasticsearch.search.SearchService; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.ShardSearchRequest; @@ -40,9 +39,10 @@ import java.util.function.BiFunction; import java.util.function.Supplier; +import static org.elasticsearch.action.search.SearchPhaseController.getTopDocsSize; import static org.elasticsearch.search.internal.SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO; -final class SearchQueryThenFetchAsyncAction extends AbstractSearchAsyncAction { +class SearchQueryThenFetchAsyncAction extends AbstractSearchAsyncAction { private final SearchPhaseController searchPhaseController; private final Supplier topDocsSupplier; @@ -61,7 +61,7 @@ final class SearchQueryThenFetchAsyncAction extends AbstractSearchAsyncAction getTopDocsSupplier(SearchRequest request, SearchPhaseResults searchPhaseResults) { + private Supplier getBufferTopDocsSupplier(SearchRequest request, + SearchPhaseResults searchPhaseResults) { if (searchPhaseResults instanceof SearchPhaseController.QueryPhaseResultConsumer == false) { return () -> null; } @@ -118,13 +119,4 @@ private Supplier getTopDocsSupplier(SearchRequest request, SearchPhaseR } return ((SearchPhaseController.QueryPhaseResultConsumer) searchPhaseResults)::getBufferTopDocs; } - - static int getTopDocsSize(SearchRequest request) { - if (request.source() == null) { - return SearchService.DEFAULT_SIZE; - } - SearchSourceBuilder source = request.source(); - return (source.size() == -1 ? SearchService.DEFAULT_SIZE : source.size()) + - (source.from() == -1 ? SearchService.DEFAULT_FROM : source.from()); - } } diff --git a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java index 9842413e9b166..f472b89cde95b 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java +++ b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java @@ -295,6 +295,12 @@ public String preference() { return preference; } + /** + * Sets the bottom sort values that can be used by the searcher to filter documents + * that are after it. This value is computed by coordinating nodes that throttles the + * query phase. After a partial merge of successful shards the sort values of the + * bottom top document are passed as an hint on subsequent shard requests. + */ public void setRawBottomSortValues(Object[] values) { this.rawBottomSortValues = values; } @@ -358,6 +364,8 @@ public Rewriteable rewrite(QueryRewriteContext ctx) throws IOException { QueryShardContext shardContext = ctx.convertToShardContext(); FieldSortBuilder primarySort = FieldSortBuilder.getPrimaryFieldSortOrNull(newSource); + // checks if the bottom sort values are guaranteed to be more competitive than all the documents + // contained in the shard if (shardContext != null && primarySort != null && primarySort.isBottomSortWithinShard(shardContext, request.getRawBottomSortValues()) == false) { diff --git a/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java b/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java index a4757b7bb587b..0e1e4c6b4121c 100644 --- a/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java @@ -381,7 +381,8 @@ public boolean canRewriteToMatchNone() { } /** - * Return true if the + * Returns whether all values of the given {@link QueryShardContext#getIndexReader()} are within the + * primary sort value provided in the rawBottomSortValues. */ public boolean isBottomSortWithinShard(QueryShardContext context, Object[] rawBottomSortValues) { @@ -403,7 +404,7 @@ public boolean isBottomSortWithinShard(QueryShardContext context, Object minValue = order() == SortOrder.DESC ? rawBottomSortValues[0] : null; Object maxValue = order() == SortOrder.DESC ? null : rawBottomSortValues[0]; try { - MappedFieldType.Relation relation = fieldType.isFieldWithinQuery(context.searcher().getIndexReader(), minValue, maxValue, + MappedFieldType.Relation relation = fieldType.isFieldWithinQuery(context.getIndexReader(), minValue, maxValue, true, true, null, DEFAULT_DATE_TIME_FORMATTER.toDateMathParser(), context); return relation != MappedFieldType.Relation.DISJOINT; } catch (Exception exc) { diff --git a/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java b/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java index b91ee4a1880a7..4b021bdb3e7be 100644 --- a/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java @@ -1,3 +1,22 @@ +/* + * 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.lucene.search.FieldDoc; @@ -5,7 +24,6 @@ import org.apache.lucene.search.TopFieldDocs; import org.apache.lucene.search.TotalHits; import org.elasticsearch.Version; -import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.GroupShardsIterator; @@ -57,11 +75,12 @@ public void sendExecuteQuery(Transport.Connection connection, ShardSearchRequest SearchTask task, SearchActionListener listener) { int shardId = request.shardId().id(); if (request.getRawBottomSortValues() != null) { + assertTrue(request.canReturnNullResponseIfMatchNoDocs()); assertNotEquals(shardId, (int) request.getRawBottomSortValues()[0]); numWithTopDocs.incrementAndGet(); } QuerySearchResult queryResult = new QuerySearchResult(123, - new SearchShardTarget("node1", new ShardId("test", "na", shardId), null, OriginalIndices.NONE)); + new SearchShardTarget("node1", new ShardId("idx", "na", shardId), null, OriginalIndices.NONE)); SortField sortField = new SortField("timestamp", SortField.Type.LONG); queryResult.topDocs(new TopDocsAndMaxScore(new TopFieldDocs( new TotalHits(1, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO), @@ -84,7 +103,7 @@ public void sendExecuteQuery(Transport.Connection connection, ShardSearchRequest searchRequest.setBatchedReduceSize(2); searchRequest.source(new SearchSourceBuilder() .size(1) - .trackTotalHitsUpTo(1) + .trackTotalHitsUpTo(2) .sort(SortBuilders.fieldSort("timestamp"))); searchRequest.allowPartialSearchResults(false); SearchPhaseController controller = new SearchPhaseController((b) -> @@ -94,29 +113,25 @@ public void sendExecuteQuery(Transport.Connection connection, ShardSearchRequest searchTransportService, (clusterAlias, node) -> lookup.get(node), Collections.singletonMap("_na_", new AliasFilter(null, Strings.EMPTY_ARRAY)), Collections.emptyMap(), Collections.emptyMap(), controller, EsExecutors.newDirectExecutorService(), searchRequest, - new ActionListener<>() { - @Override - public void onResponse(SearchResponse response) { - latch.countDown(); - } - - @Override - public void onFailure(Exception e) { - try { - throw new AssertionError(e); - } finally { + null, shardsIter, timeProvider, 0, task, + SearchResponse.Clusters.EMPTY) { + @Override + protected SearchPhase getNextPhase(SearchPhaseResults results, SearchPhaseContext context) { + return new SearchPhase("test") { + @Override + public void run() { latch.countDown(); } - } - }, shardsIter, timeProvider, 0, task, - SearchResponse.Clusters.EMPTY); + }; + } + }; action.start(); latch.await(); assertThat(successfulOps.get(), equalTo(numShards)); assertThat(numWithTopDocs.get(), greaterThanOrEqualTo(1)); SearchPhaseController.ReducedQueryPhase phase = action.results.reduce(); assertThat(phase.numReducePhases, greaterThanOrEqualTo(1)); - assertThat(phase.totalHits.value, equalTo(1L)); + assertThat(phase.totalHits.value, equalTo(2L)); assertThat(phase.totalHits.relation, equalTo(TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO)); assertThat(phase.sortedTopDocs.scoreDocs.length, equalTo(1)); assertThat(phase.sortedTopDocs.scoreDocs[0], instanceOf(FieldDoc.class)); From 961c2cd4aa0547272e1e50fda213de90e5f81754 Mon Sep 17 00:00:00 2001 From: jimczi Date: Tue, 4 Feb 2020 11:39:34 +0100 Subject: [PATCH 15/37] remove unrelated change --- .../search/SearchServiceTests.java | 39 ------------------- 1 file changed, 39 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java b/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java index 9c0b05b997933..c9dc231f0997b 100644 --- a/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java +++ b/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java @@ -19,23 +19,11 @@ package org.elasticsearch.search; import com.carrotsearch.hppc.IntArrayList; -import org.apache.lucene.codecs.CompoundFormat; -import org.apache.lucene.codecs.FilterCodec; -import org.apache.lucene.codecs.PostingsFormat; -import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat; -import org.apache.lucene.codecs.lucene70.Lucene70Codec; -import org.apache.lucene.document.Document; -import org.apache.lucene.document.Field; -import org.apache.lucene.document.TextField; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.FilterDirectoryReader; -import org.apache.lucene.index.IndexWriter; -import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.LeafReader; import org.apache.lucene.search.Query; import org.apache.lucene.store.AlreadyClosedException; -import org.apache.lucene.store.Directory; -import org.apache.lucene.store.RAMDirectory; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.OriginalIndices; @@ -910,31 +898,4 @@ public void onFailure(Exception e) { latch.await(); } } - - public void test() throws IOException { - IndexWriterConfig config = new IndexWriterConfig(); - Lucene70Codec codec = new Lucene70Codec(); - Lucene50PostingsFormat pf = new Lucene50PostingsFormat(); - config.setCodec(new FilterCodec("lucene7", codec) { - @Override - public CompoundFormat compoundFormat() { - return super.compoundFormat(); - } - - @Override - public PostingsFormat postingsFormat() { - return pf; - } - }); - Directory dir = new RAMDirectory(); - IndexWriter writer = new IndexWriter(dir, config); - for (int i = 0; i < 10; i++) { - Document doc = new Document(); - doc.add(new TextField("text", "hello", Field.Store.YES)); - writer.addDocument(doc); - } - writer.commit(); - writer.close(); - dir.close(); - } } From af204212f937a19f9b92df50034a59e76fdaa06a Mon Sep 17 00:00:00 2001 From: jimczi Date: Thu, 6 Feb 2020 09:40:23 +0100 Subject: [PATCH 16/37] fix last merge --- .../search/internal/ShardSearchRequest.java | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java index 06fe4dfad97e4..fc12578c76d59 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java +++ b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java @@ -50,11 +50,8 @@ import org.elasticsearch.indices.InvalidAliasNameException; import org.elasticsearch.search.Scroll; import org.elasticsearch.search.builder.SearchSourceBuilder; -<<<<<<< HEAD -import org.elasticsearch.search.sort.FieldSortBuilder; -======= import org.elasticsearch.search.query.QuerySearchResult; ->>>>>>> master +import org.elasticsearch.search.sort.FieldSortBuilder; import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskId; import org.elasticsearch.transport.TransportRequest; @@ -105,7 +102,7 @@ public ShardSearchRequest(OriginalIndices originalIndices, shardId, numberOfShards, searchRequest.searchType(), - searchRequest.source() != null ? searchRequest.source().shallowCopy() : null, + searchRequest.source(), searchRequest.requestCache(), aliasFilter, indexBoost, @@ -186,8 +183,6 @@ public ShardSearchRequest(StreamInput in) throws IOException { } else { canReturnNullResponseIfMatchNoDocs = false; rawBottomSortValues = null; - } else { - canReturnNullResponseIfMatchNoDocs = false; } originalIndices = OriginalIndices.readOriginalIndices(in); } From 291f7427ea7d4d96b0c36857366a15956934a9aa Mon Sep 17 00:00:00 2001 From: jimczi Date: Thu, 6 Feb 2020 12:07:13 +0100 Subject: [PATCH 17/37] fix rest test --- .../rest-api-spec/test/search/140_pre_filter_search_shards.yml | 1 - 1 file changed, 1 deletion(-) 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 bf425dfe64091..c98f3c08119b6 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 @@ -222,5 +222,4 @@ setup: - length: { hits.hits: 1 } - match: {hits.hits.0._id: "3" } - length: { aggregations.idx_terms.buckets: 3 } - - match: { hits.total: 0 } - length: { aggregations.idx_terms.buckets: 0 } From 71876e4253d3bac8f2acab736aaa74ba35edf310 Mon Sep 17 00:00:00 2001 From: jimczi Date: Thu, 6 Feb 2020 12:57:00 +0100 Subject: [PATCH 18/37] another fix --- .../rest-api-spec/test/search/140_pre_filter_search_shards.yml | 1 - 1 file changed, 1 deletion(-) 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 c98f3c08119b6..7e3b105bea700 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 @@ -222,4 +222,3 @@ setup: - length: { hits.hits: 1 } - match: {hits.hits.0._id: "3" } - length: { aggregations.idx_terms.buckets: 3 } - - length: { aggregations.idx_terms.buckets: 0 } From db4b44f855942177e4052c26ad46e2c486907f92 Mon Sep 17 00:00:00 2001 From: jimczi Date: Mon, 24 Feb 2020 12:19:45 +0100 Subject: [PATCH 19/37] address review --- .../elasticsearch/action/search/SearchPhaseController.java | 4 +++- .../action/search/SearchQueryThenFetchAsyncAction.java | 3 +-- .../java/org/elasticsearch/search/sort/FieldSortBuilder.java | 3 +-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java index e3aeecd4f5be9..81c3a9d5bc971 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java @@ -686,7 +686,9 @@ private synchronized void consumeInternal(QuerySearchResult querySearchResult) { public TopDocs getBufferTopDocs() { if (hasTopDocs) { synchronized (this) { - return topDocsBuffer[0]; + if (numReducePhases > 0) { + return topDocsBuffer[0]; + } } } return null; diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index fa901ea8c1aea..846912adf0339 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -110,9 +110,8 @@ private Supplier getBufferTopDocsSupplier(SearchRequest request, if (searchPhaseResults instanceof SearchPhaseController.QueryPhaseResultConsumer == false) { return () -> null; } - int size = getTopDocsSize(request); FieldSortBuilder fieldSort = FieldSortBuilder.getPrimaryFieldSortOrNull(request.source()); - if (size == 0 + if (topDocsSize == 0 || fieldSort == null || fieldSort.canRewriteToMatchNone() == false) { return () -> null; diff --git a/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java b/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java index a9066486fc9ea..fdf413ad5dec2 100644 --- a/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java @@ -360,8 +360,7 @@ public boolean canRewriteToMatchNone() { * Returns whether all values of the given {@link QueryShardContext#getIndexReader()} are within the * primary sort value provided in the rawBottomSortValues. */ - public boolean isBottomSortWithinShard(QueryShardContext context, - Object[] rawBottomSortValues) { + public boolean isBottomSortWithinShard(QueryShardContext context, Object[] rawBottomSortValues) { if (rawBottomSortValues == null || rawBottomSortValues.length == 0) { return true; } From bd927bd48fe359c2d80c43abf778260f768740f4 Mon Sep 17 00:00:00 2001 From: jimczi Date: Mon, 24 Feb 2020 12:54:07 +0100 Subject: [PATCH 20/37] adapt serialization version before backport --- .../elasticsearch/search/internal/ShardSearchRequest.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java index 0ab89081572f4..933b0a2681836 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java +++ b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java @@ -179,9 +179,12 @@ public ShardSearchRequest(StreamInput in) throws IOException { preference = in.readOptionalString(); if (in.getVersion().onOrAfter(Version.V_7_7_0)) { canReturnNullResponseIfMatchNoDocs = in.readBoolean(); - rawBottomSortValues = in.readOptionalArray(Lucene::readSortValue, Object[]::new); } else { canReturnNullResponseIfMatchNoDocs = false; + } + if (in.getVersion().onOrAfter(Version.V_8_0_0)) { + rawBottomSortValues = in.readOptionalArray(Lucene::readSortValue, Object[]::new); + } else { rawBottomSortValues = null; } originalIndices = OriginalIndices.readOriginalIndices(in); @@ -220,6 +223,8 @@ protected final void innerWriteTo(StreamOutput out, boolean asKey) throws IOExce } if (out.getVersion().onOrAfter(Version.V_7_7_0)) { out.writeBoolean(canReturnNullResponseIfMatchNoDocs); + } + if (out.getVersion().onOrAfter(Version.V_8_0_0)) { out.writeOptionalArray(Lucene::writeSortValue, rawBottomSortValues); } } From bf8ebde8430f12d9698f72e8d3ca54e61070993a Mon Sep 17 00:00:00 2001 From: jimczi Date: Fri, 28 Feb 2020 15:57:47 +0100 Subject: [PATCH 21/37] address review --- .../action/search/SearchPhaseController.java | 18 ++-- .../SearchQueryThenFetchAsyncAction.java | 82 +++++++++++++------ .../SearchQueryThenFetchAsyncActionTests.java | 7 +- 3 files changed, 70 insertions(+), 37 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java index 81c3a9d5bc971..c0380639bd13d 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java @@ -683,25 +683,19 @@ private synchronized void consumeInternal(QuerySearchResult querySearchResult) { processedShards[querySearchResult.getShardIndex()] = querySearchResult.getSearchShardTarget(); } - public TopDocs getBufferTopDocs() { + private synchronized List getRemainingAggs() { + return hasAggs ? Arrays.asList(aggsBuffer).subList(0, index) : null; + } + + public List getRemainingTopDocs() { if (hasTopDocs) { synchronized (this) { - if (numReducePhases > 0) { - return topDocsBuffer[0]; - } + return index == 0 ? Collections.emptyList() : Arrays.asList(topDocsBuffer).subList(0, index); } } return null; } - private synchronized List getRemainingAggs() { - return hasAggs ? Arrays.asList(aggsBuffer).subList(0, index) : null; - } - - private synchronized List getRemainingTopDocs() { - return hasTopDocs ? Arrays.asList(topDocsBuffer).subList(0, index) : null; - } - @Override public ReducedQueryPhase reduce() { ReducedQueryPhase reducePhase = controller.reducedQueryPhase(results.asList(), diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 846912adf0339..376aba76d46f2 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -20,24 +20,28 @@ package org.elasticsearch.action.search; import org.apache.logging.log4j.Logger; +import org.apache.lucene.search.FieldComparator; import org.apache.lucene.search.FieldDoc; -import org.apache.lucene.search.TopDocs; +import org.apache.lucene.search.SortField; import org.apache.lucene.search.TopFieldDocs; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.search.SearchPhaseController.QueryPhaseResultConsumer; import org.elasticsearch.cluster.routing.GroupShardsIterator; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.internal.AliasFilter; +import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.ShardSearchRequest; import org.elasticsearch.search.sort.FieldSortBuilder; import org.elasticsearch.transport.Transport; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.Executor; import java.util.function.BiFunction; -import java.util.function.Supplier; +import java.util.stream.Collectors; import static org.elasticsearch.action.search.SearchPhaseController.getTopDocsSize; import static org.elasticsearch.search.internal.SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO; @@ -45,9 +49,9 @@ class SearchQueryThenFetchAsyncAction extends AbstractSearchAsyncAction { private final SearchPhaseController searchPhaseController; - private final Supplier topDocsSupplier; private final int topDocsSize; private final SearchProgressListener progressListener; + private final QueryPhaseResultConsumer resultConsumer; SearchQueryThenFetchAsyncAction(final Logger logger, final SearchTransportService searchTransportService, final BiFunction nodeIdToConnection, final Map aliasFilter, @@ -61,7 +65,7 @@ class SearchQueryThenFetchAsyncAction extends AbstractSearchAsyncAction listener) { - ShardSearchRequest request = rewriteShardRequest(buildShardSearchRequest(shardIt)); + ShardSearchRequest request = buildShardSearchRequestInternal(shardIt); getSearchTransport().sendExecuteQuery(getConnection(shardIt.getClusterAlias(), shard.currentNodeId()), request, getTask(), listener); } @@ -86,36 +90,66 @@ protected SearchPhase getNextPhase(final SearchPhaseResults r return new FetchSearchPhase(results, searchPhaseController, context); } - ShardSearchRequest rewriteShardRequest(ShardSearchRequest request) { - TopDocs topDocs = topDocsSupplier.get(); - if (topDocs != null && topDocs instanceof TopFieldDocs) { - SearchSourceBuilder source = request.source(); - int trackTotalHits = source.trackTotalHitsUpTo() == null ? DEFAULT_TRACK_TOTAL_HITS_UP_TO : - source.trackTotalHitsUpTo(); - if (topDocs.totalHits.value >= trackTotalHits) { - SearchSourceBuilder newSource = source.shallowCopy(); - newSource.trackTotalHits(false); - if (topDocs.scoreDocs.length >= topDocsSize) { - FieldDoc bottomDoc = (FieldDoc) topDocs.scoreDocs[topDocs.scoreDocs.length-1]; - request.setRawBottomSortValues(bottomDoc.fields); + ShardSearchRequest buildShardSearchRequestInternal(SearchShardIterator shardIt) { + ShardSearchRequest request = super.buildShardSearchRequest(shardIt); + if (resultConsumer == null) { + return request; + } + List topDocsList = resultConsumer.getRemainingTopDocs().stream() + .filter(obj -> obj instanceof TopFieldDocs) + .map(obj -> (TopFieldDocs) obj) + .collect(Collectors.toList()); + if (topDocsList.isEmpty()) { + return request; + } + SearchSourceBuilder source = request.source(); + int trackTotalHits = source.trackTotalHitsUpTo() == null ? DEFAULT_TRACK_TOTAL_HITS_UP_TO : source.trackTotalHitsUpTo(); + long totalHits = -1; + FieldDoc bestBottom = null; + SortField primarySort = topDocsList.get(0).fields[0]; + FieldComparator fieldComparator = primarySort.getComparator(1, 0); + int reverseMul = primarySort.getReverse() ? -1 : 1; + // we don't want to perform a costly merge to find the best bottom field doc so + // we just check pick the best bottom document among the available buffer. This + // means that we don't have the true-best bottom but this avoids running a partial + // merge too often. + for (TopFieldDocs topDocs : topDocsList) { + totalHits += topDocs.totalHits.value; + if (topDocs.scoreDocs.length == topDocsSize) { + FieldDoc cand = (FieldDoc) topDocs.scoreDocs[topDocsSize-1]; + if (bestBottom == null || + fieldComparator.compareValues(cand.fields[0], cand.fields[0]) * reverseMul > 0) { + bestBottom = cand; } - request.source(newSource); } } + if (trackTotalHits != SearchContext.TRACK_TOTAL_HITS_ACCURATE && totalHits >= trackTotalHits) { + request.source(source.shallowCopy().trackTotalHits(false)); + } + if (bestBottom != null) { + request.setRawBottomSortValues(bestBottom.fields); + } return request; } - private Supplier getBufferTopDocsSupplier(SearchRequest request, - SearchPhaseResults searchPhaseResults) { - if (searchPhaseResults instanceof SearchPhaseController.QueryPhaseResultConsumer == false) { - return () -> null; + /** + * Returns a result consumer that exposes the buffer of partially reduced top docs + * if the primary sort can rewrite to match none. See {@link ShardSearchRequest#getRewriteable()} + * for more details. + */ + private static QueryPhaseResultConsumer getResultConsumer(SearchRequest request, + int topDocsSize, + SearchPhaseResults results) { + if (results instanceof SearchPhaseController.QueryPhaseResultConsumer == false) { + return null; } FieldSortBuilder fieldSort = FieldSortBuilder.getPrimaryFieldSortOrNull(request.source()); if (topDocsSize == 0 || fieldSort == null || fieldSort.canRewriteToMatchNone() == false) { - return () -> null; + return null; } - return ((SearchPhaseController.QueryPhaseResultConsumer) searchPhaseResults)::getBufferTopDocs; + + return (QueryPhaseResultConsumer) results; } } diff --git a/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java b/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java index 4b021bdb3e7be..297f83015bac8 100644 --- a/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java @@ -48,6 +48,7 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import static org.hamcrest.Matchers.equalTo; @@ -69,13 +70,16 @@ public void testBottomFieldSort() throws InterruptedException { int numConcurrent = randomIntBetween(1, 4); AtomicInteger numWithTopDocs = new AtomicInteger(); AtomicInteger successfulOps = new AtomicInteger(); + AtomicBoolean canReturnNullResponse = new AtomicBoolean(false); SearchTransportService searchTransportService = new SearchTransportService(null, null) { @Override public void sendExecuteQuery(Transport.Connection connection, ShardSearchRequest request, SearchTask task, SearchActionListener listener) { int shardId = request.shardId().id(); + if (request.canReturnNullResponseIfMatchNoDocs()) { + canReturnNullResponse.set(true); + } if (request.getRawBottomSortValues() != null) { - assertTrue(request.canReturnNullResponseIfMatchNoDocs()); assertNotEquals(shardId, (int) request.getRawBottomSortValues()[0]); numWithTopDocs.incrementAndGet(); } @@ -128,6 +132,7 @@ public void run() { action.start(); latch.await(); assertThat(successfulOps.get(), equalTo(numShards)); + assertTrue(canReturnNullResponse.get()); assertThat(numWithTopDocs.get(), greaterThanOrEqualTo(1)); SearchPhaseController.ReducedQueryPhase phase = action.results.reduce(); assertThat(phase.numReducePhases, greaterThanOrEqualTo(1)); From 2e198658bf44e2245e159d2735c96c1c13cd4bc6 Mon Sep 17 00:00:00 2001 From: jimczi Date: Fri, 28 Feb 2020 17:12:59 +0100 Subject: [PATCH 22/37] fix comparator --- .../action/search/SearchQueryThenFetchAsyncAction.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 376aba76d46f2..95be88632c812 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -118,7 +118,7 @@ ShardSearchRequest buildShardSearchRequestInternal(SearchShardIterator shardIt) if (topDocs.scoreDocs.length == topDocsSize) { FieldDoc cand = (FieldDoc) topDocs.scoreDocs[topDocsSize-1]; if (bestBottom == null || - fieldComparator.compareValues(cand.fields[0], cand.fields[0]) * reverseMul > 0) { + fieldComparator.compareValues(cand.fields[0], bestBottom.fields[0]) * reverseMul < 0) { bestBottom = cand; } } From 8eae737035abbf59b39194d364d68cb7a33d9fae Mon Sep 17 00:00:00 2001 From: jimczi Date: Fri, 28 Feb 2020 17:15:24 +0100 Subject: [PATCH 23/37] fix topdocs size --- .../action/search/SearchQueryThenFetchAsyncAction.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 95be88632c812..70448dcc89a9b 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -104,7 +104,7 @@ ShardSearchRequest buildShardSearchRequestInternal(SearchShardIterator shardIt) } SearchSourceBuilder source = request.source(); int trackTotalHits = source.trackTotalHitsUpTo() == null ? DEFAULT_TRACK_TOTAL_HITS_UP_TO : source.trackTotalHitsUpTo(); - long totalHits = -1; + long totalHits = 0; FieldDoc bestBottom = null; SortField primarySort = topDocsList.get(0).fields[0]; FieldComparator fieldComparator = primarySort.getComparator(1, 0); From f50ec90eaf3e66fbdc20c028190c7e20df1cdd8f Mon Sep 17 00:00:00 2001 From: jimczi Date: Tue, 3 Mar 2020 13:16:10 +0100 Subject: [PATCH 24/37] iter --- .../search/AbstractSearchAsyncAction.java | 4 +- .../action/search/SearchPhaseController.java | 26 +--- .../SearchQueryThenFetchAsyncAction.java | 117 ++++++++---------- .../action/search/SearchRequest.java | 13 ++ .../search/internal/ShardSearchRequest.java | 10 +- .../search/sort/FieldSortBuilder.java | 14 +-- .../search/sort/FieldSortBuilderTests.java | 22 ++-- 7 files changed, 99 insertions(+), 107 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index a7ee84775e00d..7464a16722a1d 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -76,7 +76,7 @@ abstract class AbstractSearchAsyncAction exten **/ private final BiFunction nodeIdToConnection; private final SearchTask task; - protected final SearchPhaseResults results; + final SearchPhaseResults results; private final long clusterStateVersion; private final Map aliasFilter; private final Map concreteIndexBoosts; @@ -464,7 +464,7 @@ public final void onShardFailure(final int shardIndex, @Nullable SearchShardTarg * @param result the result returned form the shard * @param shardIt the shard iterator */ - private void onShardResult(Result result, SearchShardIterator shardIt) { + protected void onShardResult(Result result, SearchShardIterator shardIt) { assert result.getShardIndex() != -1 : "shard index is not set"; assert result.getSearchShardTarget() != null : "search shard target must not be null"; successfulOps.incrementAndGet(); diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java index c0380639bd13d..52b886e107b59 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java @@ -611,9 +611,9 @@ static final class QueryPhaseResultConsumer extends ArraySearchPhaseResults= 2 if there is more than one expected result"); @@ -687,13 +687,8 @@ private synchronized List getRemainingAggs() { return hasAggs ? Arrays.asList(aggsBuffer).subList(0, index) : null; } - public List getRemainingTopDocs() { - if (hasTopDocs) { - synchronized (this) { - return index == 0 ? Collections.emptyList() : Arrays.asList(topDocsBuffer).subList(0, index); - } - } - return null; + private synchronized List getRemainingTopDocs() { + return hasTopDocs ? Arrays.asList(topDocsBuffer).subList(0, index) : null; } @Override @@ -715,15 +710,6 @@ int getNumBuffered() { int getNumReducePhases() { return numReducePhases; } } - private int resolveTrackTotalHits(SearchRequest request) { - if (request.scroll() != null) { - // no matter what the value of track_total_hits is - return SearchContext.TRACK_TOTAL_HITS_ACCURATE; - } - return request.source() == null ? SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO : request.source().trackTotalHitsUpTo() == null ? - SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO : request.source().trackTotalHitsUpTo(); - } - /** * Returns a new ArraySearchPhaseResults instance. This might return an instance that reduces search responses incrementally. */ @@ -734,7 +720,7 @@ ArraySearchPhaseResults newSearchPhaseResults(SearchProgressL boolean isScrollRequest = request.scroll() != null; final boolean hasAggs = source != null && source.aggregations() != null; final boolean hasTopDocs = source == null || source.size() != 0; - final int trackTotalHitsUpTo = resolveTrackTotalHits(request); + final int trackTotalHitsUpTo = request.resolveTrackTotalHitsUpTo(); if (isScrollRequest == false && (hasAggs || hasTopDocs)) { // no incremental reduce if scroll is used - we only hit a single shard or sometimes more... if (request.getBatchedReduceSize() < numShards) { diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 70448dcc89a9b..3e3ba6249a103 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -20,12 +20,11 @@ package org.elasticsearch.action.search; import org.apache.logging.log4j.Logger; -import org.apache.lucene.search.FieldComparator; import org.apache.lucene.search.FieldDoc; -import org.apache.lucene.search.SortField; +import org.apache.lucene.search.ScoreDoc; +import org.apache.lucene.search.Sort; import org.apache.lucene.search.TopFieldDocs; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.search.SearchPhaseController.QueryPhaseResultConsumer; import org.elasticsearch.cluster.routing.GroupShardsIterator; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.search.SearchPhaseResult; @@ -33,25 +32,27 @@ import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.ShardSearchRequest; +import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.search.sort.FieldSortBuilder; import org.elasticsearch.transport.Transport; -import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.Executor; import java.util.function.BiFunction; -import java.util.stream.Collectors; import static org.elasticsearch.action.search.SearchPhaseController.getTopDocsSize; -import static org.elasticsearch.search.internal.SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO; class SearchQueryThenFetchAsyncAction extends AbstractSearchAsyncAction { private final SearchPhaseController searchPhaseController; - private final int topDocsSize; private final SearchProgressListener progressListener; - private final QueryPhaseResultConsumer resultConsumer; + + // informations to track the best bottom top doc globally. + private final int topDocsSize; + private final int trackTotalHitsUpTo; + private final boolean hasPrimaryFieldSort; + private volatile TopFieldDocs bottomTopDocs; SearchQueryThenFetchAsyncAction(final Logger logger, final SearchTransportService searchTransportService, final BiFunction nodeIdToConnection, final Map aliasFilter, @@ -65,7 +66,8 @@ class SearchQueryThenFetchAsyncAction extends AbstractSearchAsyncAction listener) { - ShardSearchRequest request = buildShardSearchRequestInternal(shardIt); + + ShardSearchRequest request = rewriteShardSearchRequest(super.buildShardSearchRequest(shardIt)); getSearchTransport().sendExecuteQuery(getConnection(shardIt.getClusterAlias(), shard.currentNodeId()), request, getTask(), listener); } @@ -85,71 +88,59 @@ protected void onShardGroupFailure(int shardIndex, Exception exc) { progressListener.notifyQueryFailure(shardIndex, exc); } + @Override + protected void onShardResult(SearchPhaseResult result, SearchShardIterator shardIt) { + mergeTopDocs(result.queryResult()); + super.onShardResult(result, shardIt); + } + @Override protected SearchPhase getNextPhase(final SearchPhaseResults results, final SearchPhaseContext context) { return new FetchSearchPhase(results, searchPhaseController, context); } - ShardSearchRequest buildShardSearchRequestInternal(SearchShardIterator shardIt) { - ShardSearchRequest request = super.buildShardSearchRequest(shardIt); - if (resultConsumer == null) { - return request; + private synchronized void mergeTopDocs(QuerySearchResult result) { + if (result.isNull() + || hasPrimaryFieldSort == false + || result.topDocs().topDocs instanceof TopFieldDocs == false) { + return; } - List topDocsList = resultConsumer.getRemainingTopDocs().stream() - .filter(obj -> obj instanceof TopFieldDocs) - .map(obj -> (TopFieldDocs) obj) - .collect(Collectors.toList()); - if (topDocsList.isEmpty()) { - return request; + // merge the current best bottom field doc with the new query result + TopFieldDocs topDocs = (TopFieldDocs) result.topDocs().topDocs; + final ScoreDoc[] bottomDocs; + if (topDocs.scoreDocs.length == topDocsSize) { + FieldDoc bottom = (FieldDoc) topDocs.scoreDocs[topDocsSize - 1]; + bottomDocs = new FieldDoc[] { new FieldDoc(bottom.doc, bottom.score, bottom.fields, result.getShardIndex()) }; + bottomDocs[0].shardIndex = result.getShardIndex(); + } else { + bottomDocs = new ScoreDoc[0]; } - SearchSourceBuilder source = request.source(); - int trackTotalHits = source.trackTotalHitsUpTo() == null ? DEFAULT_TRACK_TOTAL_HITS_UP_TO : source.trackTotalHitsUpTo(); - long totalHits = 0; - FieldDoc bestBottom = null; - SortField primarySort = topDocsList.get(0).fields[0]; - FieldComparator fieldComparator = primarySort.getComparator(1, 0); - int reverseMul = primarySort.getReverse() ? -1 : 1; - // we don't want to perform a costly merge to find the best bottom field doc so - // we just check pick the best bottom document among the available buffer. This - // means that we don't have the true-best bottom but this avoids running a partial - // merge too often. - for (TopFieldDocs topDocs : topDocsList) { - totalHits += topDocs.totalHits.value; - if (topDocs.scoreDocs.length == topDocsSize) { - FieldDoc cand = (FieldDoc) topDocs.scoreDocs[topDocsSize-1]; - if (bestBottom == null || - fieldComparator.compareValues(cand.fields[0], bestBottom.fields[0]) * reverseMul < 0) { - bestBottom = cand; - } - } + TopFieldDocs toMerge = new TopFieldDocs(topDocs.totalHits, bottomDocs, topDocs.fields); + if (bottomTopDocs == null) { + bottomTopDocs = toMerge; + } else { + final Sort sort = new Sort(bottomTopDocs.fields); + bottomTopDocs = TopFieldDocs.merge(sort, 0, 1, new TopFieldDocs[]{bottomTopDocs, toMerge}, false); } - if (trackTotalHits != SearchContext.TRACK_TOTAL_HITS_ACCURATE && totalHits >= trackTotalHits) { - request.source(source.shallowCopy().trackTotalHits(false)); + } + + ShardSearchRequest rewriteShardSearchRequest(ShardSearchRequest request) { + final TopFieldDocs current = bottomTopDocs; + if (current == null) { + return request; } + + // disable tracking total hits if we already reached the required estimation. + if (trackTotalHitsUpTo != SearchContext.TRACK_TOTAL_HITS_ACCURATE + && current.totalHits.value >= trackTotalHitsUpTo) { + assert request.source() != null : "source should contain a primary sort field"; + request.source(request.source().shallowCopy().trackTotalHits(false)); + } + // set the global best bottom field doc + FieldDoc bestBottom = current.scoreDocs.length == 1 ? (FieldDoc) current.scoreDocs[0] : null; if (bestBottom != null) { request.setRawBottomSortValues(bestBottom.fields); } return request; } - - /** - * Returns a result consumer that exposes the buffer of partially reduced top docs - * if the primary sort can rewrite to match none. See {@link ShardSearchRequest#getRewriteable()} - * for more details. - */ - private static QueryPhaseResultConsumer getResultConsumer(SearchRequest request, - int topDocsSize, - SearchPhaseResults results) { - if (results instanceof SearchPhaseController.QueryPhaseResultConsumer == false) { - return null; - } - FieldSortBuilder fieldSort = FieldSortBuilder.getPrimaryFieldSortOrNull(request.source()); - if (topDocsSize == 0 - || fieldSort == null - || fieldSort.canRewriteToMatchNone() == false) { - return null; - } - - return (QueryPhaseResultConsumer) results; - } } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java b/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java index 8ba9a8c9f0bd7..328402d41f647 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java @@ -559,6 +559,19 @@ public boolean isSuggestOnly() { return source != null && source.isSuggestOnly(); } + public int resolveTrackTotalHitsUpTo() { + return resolveTrackTotalHitsUpTo(scroll, source); + } + + public static int resolveTrackTotalHitsUpTo(Scroll scroll, SearchSourceBuilder source) { + if (scroll != null) { + // no matter what the value of track_total_hits is + return SearchContext.TRACK_TOTAL_HITS_ACCURATE; + } + return source == null ? SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO : source.trackTotalHitsUpTo() == null ? + SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO : source.trackTotalHitsUpTo(); + } + @Override public Task createTask(long id, String type, String action, TaskId parentTaskId, Map headers) { // generating description in a lazy way since source can be quite big diff --git a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java index 933b0a2681836..cc99668794fff 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java +++ b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java @@ -374,14 +374,16 @@ public Rewriteable rewrite(QueryRewriteContext ctx) throws IOException { AliasFilter newAliasFilter = Rewriteable.rewrite(request.getAliasFilter(), ctx); QueryShardContext shardContext = ctx.convertToShardContext(); + FieldSortBuilder primarySort = FieldSortBuilder.getPrimaryFieldSortOrNull(newSource); - // checks if the bottom sort values are guaranteed to be more competitive than all the documents - // contained in the shard if (shardContext != null && primarySort != null - && primarySort.isBottomSortWithinShard(shardContext, request.getRawBottomSortValues()) == false) { + && primarySort.isBottomSortShardDisjoint(shardContext, request.getRawBottomSortValues())) { + assert newSource != null : "source should contain a primary sort field"; newSource = newSource.shallowCopy(); - if (newSource.trackTotalHitsUpTo() == TRACK_TOTAL_HITS_DISABLED + int trackTotalHitsUpTo = SearchRequest.resolveTrackTotalHitsUpTo(request.scroll, request.source); + if (trackTotalHitsUpTo == TRACK_TOTAL_HITS_DISABLED + && newSource.suggest() == null && newSource.aggregations() == null) { newSource.query(new MatchNoneQueryBuilder()); } else { diff --git a/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java b/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java index db6871edd5edb..323cc9a09645b 100644 --- a/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java @@ -357,16 +357,16 @@ public boolean canRewriteToMatchNone() { } /** - * Returns whether all values of the given {@link QueryShardContext#getIndexReader()} are within the + * Returns whether some values of the given {@link QueryShardContext#getIndexReader()} are within the * primary sort value provided in the rawBottomSortValues. */ - public boolean isBottomSortWithinShard(QueryShardContext context, Object[] rawBottomSortValues) { + public boolean isBottomSortShardDisjoint(QueryShardContext context, Object[] rawBottomSortValues) { if (rawBottomSortValues == null || rawBottomSortValues.length == 0) { - return true; + return false; } if (canRewriteToMatchNone() == false) { - return true; + return false; } MappedFieldType fieldType = context.fieldMapper(fieldName); if (fieldType == null) { @@ -374,16 +374,16 @@ public boolean isBottomSortWithinShard(QueryShardContext context, Object[] rawBo return false; } if (fieldType.indexOptions() == IndexOptions.NONE) { - return true; + return false; } Object minValue = order() == SortOrder.DESC ? rawBottomSortValues[0] : null; Object maxValue = order() == SortOrder.DESC ? null : rawBottomSortValues[0]; try { MappedFieldType.Relation relation = fieldType.isFieldWithinQuery(context.getIndexReader(), minValue, maxValue, true, true, null, DEFAULT_DATE_TIME_FORMATTER.toDateMathParser(), context); - return relation != MappedFieldType.Relation.DISJOINT; + return relation == MappedFieldType.Relation.DISJOINT; } catch (Exception exc) { - return true; + return false; } } diff --git a/server/src/test/java/org/elasticsearch/search/sort/FieldSortBuilderTests.java b/server/src/test/java/org/elasticsearch/search/sort/FieldSortBuilderTests.java index f41543c9c542b..6fe340d67471b 100644 --- a/server/src/test/java/org/elasticsearch/search/sort/FieldSortBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/search/sort/FieldSortBuilderTests.java @@ -569,7 +569,7 @@ public void testGetMaxKeywordValue() throws IOException { } } - public void testIsBottomSortWithinShard() throws Exception { + public void testIsBottomSortShardDisjoint() throws Exception { try (Directory dir = newDirectory()) { int numDocs = randomIntBetween(5, 10); long maxValue = -1; @@ -578,7 +578,7 @@ public void testIsBottomSortWithinShard() throws Exception { FieldSortBuilder fieldSort = SortBuilders.fieldSort("custom-date"); try (DirectoryReader reader = writer.getReader()) { QueryShardContext context = createMockShardContext(new IndexSearcher(reader)); - assertFalse(fieldSort.isBottomSortWithinShard(context, new Object[] { 0L })); + assertTrue(fieldSort.isBottomSortShardDisjoint(context, new Object[] { 0L })); } for (int i = 0; i < numDocs; i++) { Document doc = new Document(); @@ -591,19 +591,19 @@ public void testIsBottomSortWithinShard() throws Exception { } try (DirectoryReader reader = writer.getReader()) { QueryShardContext context = createMockShardContext(new IndexSearcher(reader)); - assertTrue(fieldSort.isBottomSortWithinShard(context, null)); - assertTrue(fieldSort.isBottomSortWithinShard(context, new Object[] { minValue })); - assertFalse(fieldSort.isBottomSortWithinShard(context, new Object[] { minValue-1 })); - assertTrue(fieldSort.isBottomSortWithinShard(context, new Object[] { minValue+1 })); + assertFalse(fieldSort.isBottomSortShardDisjoint(context, null)); + assertFalse(fieldSort.isBottomSortShardDisjoint(context, new Object[] { minValue })); + assertTrue(fieldSort.isBottomSortShardDisjoint(context, new Object[] { minValue-1 })); + assertFalse(fieldSort.isBottomSortShardDisjoint(context, new Object[] { minValue+1 })); fieldSort.order(SortOrder.DESC); - assertFalse(fieldSort.isBottomSortWithinShard(context, new Object[] { maxValue+1 })); - assertTrue(fieldSort.isBottomSortWithinShard(context, new Object[] { maxValue })); - assertTrue(fieldSort.isBottomSortWithinShard(context, new Object[] { minValue })); + assertTrue(fieldSort.isBottomSortShardDisjoint(context, new Object[] { maxValue+1 })); + assertFalse(fieldSort.isBottomSortShardDisjoint(context, new Object[] { maxValue })); + assertFalse(fieldSort.isBottomSortShardDisjoint(context, new Object[] { minValue })); fieldSort.setNestedSort(new NestedSortBuilder("empty")); - assertTrue(fieldSort.isBottomSortWithinShard(context, new Object[] { minValue-1 })); + assertFalse(fieldSort.isBottomSortShardDisjoint(context, new Object[] { minValue-1 })); fieldSort.setNestedSort(null); fieldSort.missing("100"); - assertTrue(fieldSort.isBottomSortWithinShard(context, new Object[] { maxValue+1 })); + assertFalse(fieldSort.isBottomSortShardDisjoint(context, new Object[] { maxValue+1 })); } } } From 4ec6a6caee3981475f0b8b3bf9da3815d19b2c86 Mon Sep 17 00:00:00 2001 From: jimczi Date: Tue, 3 Mar 2020 17:47:43 +0100 Subject: [PATCH 25/37] fix final merge when some shard responses have their size artifically rewrote to 0 --- .../action/search/SearchPhaseController.java | 3 +- .../SearchQueryThenFetchAsyncAction.java | 31 +++++++++++-------- 2 files changed, 20 insertions(+), 14 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java index 52b886e107b59..17827d58ed72d 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java @@ -467,7 +467,8 @@ private ReducedQueryPhase reducedQueryPhase(Collection> suggestion : result.suggest()) { diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 3e3ba6249a103..30e9dca0f3f86 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -24,6 +24,7 @@ import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.Sort; import org.apache.lucene.search.TopFieldDocs; +import org.apache.lucene.search.TotalHits; import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.routing.GroupShardsIterator; import org.elasticsearch.cluster.routing.ShardRouting; @@ -41,6 +42,7 @@ import java.util.concurrent.Executor; import java.util.function.BiFunction; +import static org.apache.lucene.search.TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO; import static org.elasticsearch.action.search.SearchPhaseController.getTopDocsSize; class SearchQueryThenFetchAsyncAction extends AbstractSearchAsyncAction { @@ -90,7 +92,16 @@ protected void onShardGroupFailure(int shardIndex, Exception exc) { @Override protected void onShardResult(SearchPhaseResult result, SearchShardIterator shardIt) { - mergeTopDocs(result.queryResult()); + try { + QuerySearchResult queryResult = result.queryResult(); + if (queryResult.isNull() == false + && hasPrimaryFieldSort + && queryResult.topDocs().topDocs instanceof TopFieldDocs) { + mergeTopDocs((TopFieldDocs) result.queryResult().topDocs().topDocs, result.getShardIndex()); + } + } catch (Exception exc) { + getLogger().error("Failed to merge response", exc); + } super.onShardResult(result, shardIt); } @@ -99,19 +110,12 @@ protected SearchPhase getNextPhase(final SearchPhaseResults r return new FetchSearchPhase(results, searchPhaseController, context); } - private synchronized void mergeTopDocs(QuerySearchResult result) { - if (result.isNull() - || hasPrimaryFieldSort == false - || result.topDocs().topDocs instanceof TopFieldDocs == false) { - return; - } - // merge the current best bottom field doc with the new query result - TopFieldDocs topDocs = (TopFieldDocs) result.topDocs().topDocs; + // merge the current best bottom field doc with the new query result + private synchronized void mergeTopDocs(TopFieldDocs topDocs, int shardIndex) { final ScoreDoc[] bottomDocs; if (topDocs.scoreDocs.length == topDocsSize) { FieldDoc bottom = (FieldDoc) topDocs.scoreDocs[topDocsSize - 1]; - bottomDocs = new FieldDoc[] { new FieldDoc(bottom.doc, bottom.score, bottom.fields, result.getShardIndex()) }; - bottomDocs[0].shardIndex = result.getShardIndex(); + bottomDocs = new FieldDoc[] { new FieldDoc(bottom.doc, bottom.score, bottom.fields, shardIndex) }; } else { bottomDocs = new ScoreDoc[0]; } @@ -120,7 +124,7 @@ private synchronized void mergeTopDocs(QuerySearchResult result) { bottomTopDocs = toMerge; } else { final Sort sort = new Sort(bottomTopDocs.fields); - bottomTopDocs = TopFieldDocs.merge(sort, 0, 1, new TopFieldDocs[]{bottomTopDocs, toMerge}, false); + bottomTopDocs = TopFieldDocs.merge(sort, 0, 1, new TopFieldDocs[]{ bottomTopDocs, toMerge }, false); } } @@ -132,7 +136,8 @@ ShardSearchRequest rewriteShardSearchRequest(ShardSearchRequest request) { // disable tracking total hits if we already reached the required estimation. if (trackTotalHitsUpTo != SearchContext.TRACK_TOTAL_HITS_ACCURATE - && current.totalHits.value >= trackTotalHitsUpTo) { + && current.totalHits.value >= trackTotalHitsUpTo + && current.totalHits.relation == GREATER_THAN_OR_EQUAL_TO) { assert request.source() != null : "source should contain a primary sort field"; request.source(request.source().shallowCopy().trackTotalHits(false)); } From 09bfb098a2e21f2e435f6318929e04bda9808eaf Mon Sep 17 00:00:00 2001 From: jimczi Date: Tue, 3 Mar 2020 18:07:08 +0100 Subject: [PATCH 26/37] check style --- .../action/search/SearchQueryThenFetchAsyncAction.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 30e9dca0f3f86..9419cdb5b90ef 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -24,7 +24,6 @@ import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.Sort; import org.apache.lucene.search.TopFieldDocs; -import org.apache.lucene.search.TotalHits; import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.routing.GroupShardsIterator; import org.elasticsearch.cluster.routing.ShardRouting; From 5e268c6bf5ba52c844871f1f488a3dcf4a1d74e3 Mon Sep 17 00:00:00 2001 From: jimczi Date: Tue, 3 Mar 2020 20:44:57 +0100 Subject: [PATCH 27/37] return topfielddocs if the shard search request rewrites the size to 0 --- .../elasticsearch/common/lucene/Lucene.java | 1 + .../search/query/TopDocsCollectorContext.java | 20 +++++++++++++++---- 2 files changed, 17 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java index ef2816777094a..2fbdff8e6ec76 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java @@ -111,6 +111,7 @@ public class Lucene { public static final NamedAnalyzer KEYWORD_ANALYZER = new NamedAnalyzer("_keyword", AnalyzerScope.GLOBAL, new KeywordAnalyzer()); public static final ScoreDoc[] EMPTY_SCORE_DOCS = new ScoreDoc[0]; + public static final ScoreDoc[] EMPTY_FIELD_DOCS = new ScoreDoc[0]; public static final TopDocs EMPTY_TOP_DOCS = new TopDocs(new TotalHits(0, TotalHits.Relation.EQUAL_TO), EMPTY_SCORE_DOCS); diff --git a/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java b/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java index f912d1e99129a..04b3a10832102 100644 --- a/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java +++ b/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java @@ -99,6 +99,7 @@ boolean shouldRescore() { } static class EmptyTopDocsCollectorContext extends TopDocsCollectorContext { + private final Sort sort; private final Collector collector; private final Supplier hitCountSupplier; @@ -109,9 +110,13 @@ static class EmptyTopDocsCollectorContext extends TopDocsCollectorContext { * @param trackTotalHitsUpTo True if the total number of hits should be tracked * @param hasFilterCollector True if the collector chain contains a filter */ - private EmptyTopDocsCollectorContext(IndexReader reader, Query query, - int trackTotalHitsUpTo, boolean hasFilterCollector) throws IOException { + private EmptyTopDocsCollectorContext(IndexReader reader, + Query query, + @Nullable SortAndFormats sortAndFormats, + int trackTotalHitsUpTo, + boolean hasFilterCollector) throws IOException { super(REASON_SEARCH_COUNT, 0); + this.sort = sortAndFormats == null ? null : sortAndFormats.sort; if (trackTotalHitsUpTo == SearchContext.TRACK_TOTAL_HITS_DISABLED) { this.collector = new EarlyTerminatingCollector(new TotalHitCountCollector(), 0, false); // for bwc hit count is set to 0, it will be converted to -1 by the coordinating node @@ -147,7 +152,13 @@ Collector create(Collector in) { @Override void postProcess(QuerySearchResult result) { final TotalHits totalHitCount = hitCountSupplier.get(); - result.topDocs(new TopDocsAndMaxScore(new TopDocs(totalHitCount, Lucene.EMPTY_SCORE_DOCS), Float.NaN), null); + final TopDocs topDocs; + if (sort != null) { + topDocs = new TopFieldDocs(totalHitCount, Lucene.EMPTY_FIELD_DOCS, sort.getSort()); + } else { + topDocs = new TopDocs(totalHitCount, Lucene.EMPTY_SCORE_DOCS); + } + result.topDocs(new TopDocsAndMaxScore(topDocs, Float.NaN), null); } } @@ -421,7 +432,8 @@ static TopDocsCollectorContext createTopDocsCollectorContext(SearchContext searc final int totalNumDocs = Math.max(1, reader.numDocs()); if (searchContext.size() == 0) { // no matter what the value of from is - return new EmptyTopDocsCollectorContext(reader, query, searchContext.trackTotalHitsUpTo(), hasFilterCollector); + return new EmptyTopDocsCollectorContext(reader, query, searchContext.sort(), + searchContext.trackTotalHitsUpTo(), hasFilterCollector); } else if (searchContext.scrollContext() != null) { // we can disable the tracking of total hits after the initial scroll query // since the total hits is preserved in the scroll context. From ae1b2789bbc509e17c4b99b32d1a2bbb59eac6f0 Mon Sep 17 00:00:00 2001 From: jimczi Date: Mon, 9 Mar 2020 16:50:03 +0100 Subject: [PATCH 28/37] Replace raw sort values with formatted ones to handle date and date_nanos correctly --- .../search/140_pre_filter_search_shards.yml | 6 +- .../search/BottomSortValuesCollector.java | 88 +++++++ .../SearchQueryThenFetchAsyncAction.java | 63 ++--- .../elasticsearch/common/lucene/Lucene.java | 1 - .../elasticsearch/search/DocValueFormat.java | 4 + .../search/SearchSortValuesAndFormats.java | 102 ++++++++ .../search/internal/ShardSearchRequest.java | 22 +- .../search/query/TopDocsCollectorContext.java | 2 +- .../search/sort/FieldSortBuilder.java | 35 ++- .../BottomSortValuesCollectorTests.java | 230 ++++++++++++++++++ .../SearchQueryThenFetchAsyncActionTests.java | 4 +- .../SearchSortValuesAndFormatsTests.java | 80 ++++++ .../internal/ShardSearchRequestTests.java | 5 +- .../search/sort/FieldSortBuilderTests.java | 28 ++- 14 files changed, 582 insertions(+), 88 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/action/search/BottomSortValuesCollector.java create mode 100644 server/src/main/java/org/elasticsearch/search/SearchSortValuesAndFormats.java create mode 100644 server/src/test/java/org/elasticsearch/action/search/BottomSortValuesCollectorTests.java create mode 100644 server/src/test/java/org/elasticsearch/search/SearchSortValuesAndFormatsTests.java 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 7e3b105bea700..15cea782c4679 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 @@ -19,7 +19,7 @@ setup: mappings: properties: created_at: - type: date + type: date_nanos format: "yyyy-MM-dd" - do: indices.create: @@ -190,7 +190,7 @@ setup: body: "size": 1 "track_total_hits": 1 - "sort": [{ "created_at": "desc"}] + "sort": [{ "created_at": { "order": "desc", "numeric_type": "date" } }] - match: { _shards.total: 3 } - match: { _shards.successful: 3 } @@ -210,7 +210,7 @@ setup: body: "size": 1 "track_total_hits": 1 - "sort": [{ "created_at": "desc"}] + "sort": [{ "created_at": { "order": "desc", "numeric_type": "date" } }] "aggs" : { "idx_terms" : { "terms" : { "field" : "_index" } } } - match: { _shards.total: 3 } diff --git a/server/src/main/java/org/elasticsearch/action/search/BottomSortValuesCollector.java b/server/src/main/java/org/elasticsearch/action/search/BottomSortValuesCollector.java new file mode 100644 index 0000000000000..052c927e2e45a --- /dev/null +++ b/server/src/main/java/org/elasticsearch/action/search/BottomSortValuesCollector.java @@ -0,0 +1,88 @@ +/* + * 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.lucene.search.FieldComparator; +import org.apache.lucene.search.FieldDoc; +import org.apache.lucene.search.SortField; +import org.apache.lucene.search.TopFieldDocs; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.SearchSortValuesAndFormats; + +/** + * Utility class to keep track of the bottom doc's sort values in a distributed search. + */ +class BottomSortValuesCollector { + private final int topNSize; + private final FieldComparator[] comparators; + private final int[] reverseMuls; + + private volatile long totalHits; + private volatile SearchSortValuesAndFormats bottomSortValues; + + BottomSortValuesCollector(int topNSize, SortField[] sortFields) { + this.topNSize = topNSize; + this.comparators = new FieldComparator[sortFields.length]; + this.reverseMuls = new int[sortFields.length]; + for (int i = 0; i < sortFields.length; i++) { + comparators[i] = sortFields[i].getComparator(1, i); + reverseMuls[i] = sortFields[i].getReverse() ? -1 : 1; + } + } + + long getTotalHits() { + return totalHits; + } + + /** + * @return The best bottom sort values consumed so far. + */ + SearchSortValuesAndFormats getBottomSortValues() { + return bottomSortValues; + } + + synchronized void consumeTopDocs(TopFieldDocs topDocs, DocValueFormat[] sortValuesFormat) { + totalHits += topDocs.totalHits.value; + + FieldDoc shardBottomDoc = extractBottom(topDocs); + if (shardBottomDoc == null) { + return; + } + if (bottomSortValues == null + || compareValues(shardBottomDoc.fields, bottomSortValues.getRawSortValues()) < 0) { + bottomSortValues = new SearchSortValuesAndFormats(shardBottomDoc.fields, sortValuesFormat); + } + } + + private FieldDoc extractBottom(TopFieldDocs topDocs) { + return topNSize > 0 && topDocs.scoreDocs.length == topNSize ? + (FieldDoc) topDocs.scoreDocs[topNSize-1] : null; + } + + private int compareValues(Object[] v1, Object[] v2) { + for (int i = 0; i < v1.length; i++) { + int cmp = reverseMuls[i] * comparators[i].compareValues(v1[i], v2[i]); + if (cmp != 0) { + return cmp; + } + } + return 0; + } +} diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 9419cdb5b90ef..a9c78a176724d 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -20,9 +20,6 @@ package org.elasticsearch.action.search; import org.apache.logging.log4j.Logger; -import org.apache.lucene.search.FieldDoc; -import org.apache.lucene.search.ScoreDoc; -import org.apache.lucene.search.Sort; import org.apache.lucene.search.TopFieldDocs; import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.routing.GroupShardsIterator; @@ -33,7 +30,6 @@ import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.ShardSearchRequest; import org.elasticsearch.search.query.QuerySearchResult; -import org.elasticsearch.search.sort.FieldSortBuilder; import org.elasticsearch.transport.Transport; import java.util.Map; @@ -41,7 +37,6 @@ import java.util.concurrent.Executor; import java.util.function.BiFunction; -import static org.apache.lucene.search.TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO; import static org.elasticsearch.action.search.SearchPhaseController.getTopDocsSize; class SearchQueryThenFetchAsyncAction extends AbstractSearchAsyncAction { @@ -52,8 +47,7 @@ class SearchQueryThenFetchAsyncAction extends AbstractSearchAsyncAction nodeIdToConnection, final Map aliasFilter, @@ -68,7 +62,6 @@ class SearchQueryThenFetchAsyncAction extends AbstractSearchAsyncAction listener) { - ShardSearchRequest request = rewriteShardSearchRequest(super.buildShardSearchRequest(shardIt)); getSearchTransport().sendExecuteQuery(getConnection(shardIt.getClusterAlias(), shard.currentNodeId()), request, getTask(), listener); @@ -91,15 +83,17 @@ protected void onShardGroupFailure(int shardIndex, Exception exc) { @Override protected void onShardResult(SearchPhaseResult result, SearchShardIterator shardIt) { - try { - QuerySearchResult queryResult = result.queryResult(); - if (queryResult.isNull() == false - && hasPrimaryFieldSort - && queryResult.topDocs().topDocs instanceof TopFieldDocs) { - mergeTopDocs((TopFieldDocs) result.queryResult().topDocs().topDocs, result.getShardIndex()); + QuerySearchResult queryResult = result.queryResult(); + if (queryResult.isNull() == false && queryResult.topDocs().topDocs instanceof TopFieldDocs) { + TopFieldDocs topDocs = (TopFieldDocs) queryResult.topDocs().topDocs; + if (bottomSortCollector == null) { + synchronized (this) { + if (bottomSortCollector == null) { + bottomSortCollector = new BottomSortValuesCollector(topDocsSize, topDocs.fields); + } + } } - } catch (Exception exc) { - getLogger().error("Failed to merge response", exc); + bottomSortCollector.consumeTopDocs(topDocs, queryResult.sortValueFormats()); } super.onShardResult(result, shardIt); } @@ -109,41 +103,20 @@ protected SearchPhase getNextPhase(final SearchPhaseResults r return new FetchSearchPhase(results, searchPhaseController, context); } - // merge the current best bottom field doc with the new query result - private synchronized void mergeTopDocs(TopFieldDocs topDocs, int shardIndex) { - final ScoreDoc[] bottomDocs; - if (topDocs.scoreDocs.length == topDocsSize) { - FieldDoc bottom = (FieldDoc) topDocs.scoreDocs[topDocsSize - 1]; - bottomDocs = new FieldDoc[] { new FieldDoc(bottom.doc, bottom.score, bottom.fields, shardIndex) }; - } else { - bottomDocs = new ScoreDoc[0]; - } - TopFieldDocs toMerge = new TopFieldDocs(topDocs.totalHits, bottomDocs, topDocs.fields); - if (bottomTopDocs == null) { - bottomTopDocs = toMerge; - } else { - final Sort sort = new Sort(bottomTopDocs.fields); - bottomTopDocs = TopFieldDocs.merge(sort, 0, 1, new TopFieldDocs[]{ bottomTopDocs, toMerge }, false); - } - } - - ShardSearchRequest rewriteShardSearchRequest(ShardSearchRequest request) { - final TopFieldDocs current = bottomTopDocs; - if (current == null) { + private ShardSearchRequest rewriteShardSearchRequest(ShardSearchRequest request) { + if (bottomSortCollector == null) { return request; } // disable tracking total hits if we already reached the required estimation. if (trackTotalHitsUpTo != SearchContext.TRACK_TOTAL_HITS_ACCURATE - && current.totalHits.value >= trackTotalHitsUpTo - && current.totalHits.relation == GREATER_THAN_OR_EQUAL_TO) { - assert request.source() != null : "source should contain a primary sort field"; + && bottomSortCollector.getTotalHits() >= trackTotalHitsUpTo) { request.source(request.source().shallowCopy().trackTotalHits(false)); } - // set the global best bottom field doc - FieldDoc bestBottom = current.scoreDocs.length == 1 ? (FieldDoc) current.scoreDocs[0] : null; - if (bestBottom != null) { - request.setRawBottomSortValues(bestBottom.fields); + + // set the current best bottom field doc + if (bottomSortCollector.getBottomSortValues() != null) { + request.setBottomSortValues(bottomSortCollector.getBottomSortValues()); } return request; } diff --git a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java index 2fbdff8e6ec76..ef2816777094a 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java @@ -111,7 +111,6 @@ public class Lucene { public static final NamedAnalyzer KEYWORD_ANALYZER = new NamedAnalyzer("_keyword", AnalyzerScope.GLOBAL, new KeywordAnalyzer()); public static final ScoreDoc[] EMPTY_SCORE_DOCS = new ScoreDoc[0]; - public static final ScoreDoc[] EMPTY_FIELD_DOCS = new ScoreDoc[0]; public static final TopDocs EMPTY_TOP_DOCS = new TopDocs(new TotalHits(0, TotalHits.Relation.EQUAL_TO), EMPTY_SCORE_DOCS); diff --git a/server/src/main/java/org/elasticsearch/search/DocValueFormat.java b/server/src/main/java/org/elasticsearch/search/DocValueFormat.java index b09c8983c1394..b91b4ff62bc09 100644 --- a/server/src/main/java/org/elasticsearch/search/DocValueFormat.java +++ b/server/src/main/java/org/elasticsearch/search/DocValueFormat.java @@ -215,6 +215,10 @@ public void writeTo(StreamOutput out) throws IOException { out.writeVInt(resolution.ordinal()); } + public DateMathParser getDateMathParser() { + return parser; + } + @Override public String format(long value) { return formatter.format(resolution.toInstant(value).atZone(timeZone)); diff --git a/server/src/main/java/org/elasticsearch/search/SearchSortValuesAndFormats.java b/server/src/main/java/org/elasticsearch/search/SearchSortValuesAndFormats.java new file mode 100644 index 0000000000000..5127c6e56d669 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/SearchSortValuesAndFormats.java @@ -0,0 +1,102 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.lucene.Lucene; + +import java.io.IOException; +import java.util.Arrays; + +public class SearchSortValuesAndFormats implements Writeable { + private final Object[] rawSortValues; + private final Object[] formattedSortValues; + private final DocValueFormat[] sortValueFormats; + + public SearchSortValuesAndFormats(Object[] rawSortValues, DocValueFormat[] sortValueFormats) { + assert rawSortValues.length == sortValueFormats.length; + this.rawSortValues = rawSortValues; + this.sortValueFormats = sortValueFormats; + this.formattedSortValues = Arrays.copyOf(rawSortValues, rawSortValues.length); + for (int i = 0; i < rawSortValues.length; ++i) { + Object sortValue = rawSortValues[i]; + if (sortValue instanceof BytesRef) { + this.formattedSortValues[i] = sortValueFormats[i].format((BytesRef) sortValue); + } else if (sortValue instanceof Long) { + this.formattedSortValues[i] = sortValueFormats[i].format((long) sortValue); + } else if (sortValue instanceof Double) { + this.formattedSortValues[i] = sortValueFormats[i].format((double) sortValue); + } else { + this.formattedSortValues[i] = sortValue; + } + } + } + + public SearchSortValuesAndFormats(StreamInput in) throws IOException { + this.rawSortValues = in.readArray(Lucene::readSortValue, Object[]::new); + this.formattedSortValues = in.readArray(Lucene::readSortValue, Object[]::new); + this.sortValueFormats = new DocValueFormat[formattedSortValues.length]; + for (int i = 0; i < sortValueFormats.length; ++i) { + sortValueFormats[i] = in.readNamedWriteable(DocValueFormat.class); + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeArray(Lucene::writeSortValue, rawSortValues); + out.writeArray(Lucene::writeSortValue, formattedSortValues); + for (int i = 0; i < sortValueFormats.length; i++) { + out.writeNamedWriteable(sortValueFormats[i]); + } + } + + public Object[] getRawSortValues() { + return rawSortValues; + } + + public Object[] getFormattedSortValues() { + return formattedSortValues; + } + + public DocValueFormat[] getSortValueFormats() { + return sortValueFormats; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + SearchSortValuesAndFormats that = (SearchSortValuesAndFormats) o; + return Arrays.equals(rawSortValues, that.rawSortValues) && + Arrays.equals(formattedSortValues, that.formattedSortValues) && + Arrays.equals(sortValueFormats, that.sortValueFormats); + } + + @Override + public int hashCode() { + int result = Arrays.hashCode(rawSortValues); + result = 31 * result + Arrays.hashCode(formattedSortValues); + result = 31 * result + Arrays.hashCode(sortValueFormats); + return result; + } +} diff --git a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java index cc99668794fff..6c1fc7e959b40 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java +++ b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java @@ -37,7 +37,6 @@ import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.index.Index; import org.elasticsearch.index.query.BoolQueryBuilder; import org.elasticsearch.index.query.MatchNoneQueryBuilder; @@ -48,6 +47,7 @@ import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.AliasFilterParsingException; import org.elasticsearch.indices.InvalidAliasNameException; +import org.elasticsearch.search.SearchSortValuesAndFormats; import org.elasticsearch.search.Scroll; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.query.QuerySearchResult; @@ -83,7 +83,7 @@ public class ShardSearchRequest extends TransportRequest implements IndicesReque private final OriginalIndices originalIndices; private boolean canReturnNullResponseIfMatchNoDocs; - private Object[] rawBottomSortValues; + private SearchSortValuesAndFormats bottomSortValues; //these are the only mutable fields, as they are subject to rewriting private AliasFilter aliasFilter; @@ -183,9 +183,9 @@ public ShardSearchRequest(StreamInput in) throws IOException { canReturnNullResponseIfMatchNoDocs = false; } if (in.getVersion().onOrAfter(Version.V_8_0_0)) { - rawBottomSortValues = in.readOptionalArray(Lucene::readSortValue, Object[]::new); + bottomSortValues = in.readOptionalWriteable(SearchSortValuesAndFormats::new); } else { - rawBottomSortValues = null; + bottomSortValues = null; } originalIndices = OriginalIndices.readOriginalIndices(in); } @@ -225,7 +225,7 @@ protected final void innerWriteTo(StreamOutput out, boolean asKey) throws IOExce out.writeBoolean(canReturnNullResponseIfMatchNoDocs); } if (out.getVersion().onOrAfter(Version.V_8_0_0)) { - out.writeOptionalArray(Lucene::writeSortValue, rawBottomSortValues); + out.writeOptionalWriteable(bottomSortValues); } } @@ -307,12 +307,12 @@ public String preference() { * query phase. After a partial merge of successful shards the sort values of the * bottom top document are passed as an hint on subsequent shard requests. */ - public void setRawBottomSortValues(Object[] values) { - this.rawBottomSortValues = values; + public void setBottomSortValues(SearchSortValuesAndFormats values) { + this.bottomSortValues = values; } - public Object[] getRawBottomSortValues() { - return rawBottomSortValues; + public SearchSortValuesAndFormats getBottomSortValues() { + return bottomSortValues; } /** @@ -378,7 +378,7 @@ public Rewriteable rewrite(QueryRewriteContext ctx) throws IOException { FieldSortBuilder primarySort = FieldSortBuilder.getPrimaryFieldSortOrNull(newSource); if (shardContext != null && primarySort != null - && primarySort.isBottomSortShardDisjoint(shardContext, request.getRawBottomSortValues())) { + && primarySort.isBottomSortShardDisjoint(shardContext, request.getBottomSortValues())) { assert newSource != null : "source should contain a primary sort field"; newSource = newSource.shallowCopy(); int trackTotalHitsUpTo = SearchRequest.resolveTrackTotalHitsUpTo(request.scroll, request.source); @@ -390,7 +390,7 @@ public Rewriteable rewrite(QueryRewriteContext ctx) throws IOException { newSource.size(0); } request.source(newSource); - request.setRawBottomSortValues(null); + request.setBottomSortValues(null); } if (newSource == request.source() && newAliasFilter == request.getAliasFilter()) { diff --git a/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java b/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java index 04b3a10832102..3f8ef30863584 100644 --- a/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java +++ b/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java @@ -154,7 +154,7 @@ void postProcess(QuerySearchResult result) { final TotalHits totalHitCount = hitCountSupplier.get(); final TopDocs topDocs; if (sort != null) { - topDocs = new TopFieldDocs(totalHitCount, Lucene.EMPTY_FIELD_DOCS, sort.getSort()); + topDocs = new TopFieldDocs(totalHitCount, Lucene.EMPTY_SCORE_DOCS, sort.getSort()); } else { topDocs = new TopDocs(totalHitCount, Lucene.EMPTY_SCORE_DOCS); } diff --git a/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java b/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java index b5dee10240684..c892703272e5b 100644 --- a/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java @@ -30,6 +30,7 @@ import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.time.DateMathParser; import org.elasticsearch.common.time.DateUtils; import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.ObjectParser.ValueType; @@ -50,6 +51,7 @@ import org.elasticsearch.index.query.QueryRewriteContext; import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.query.QueryShardException; +import org.elasticsearch.search.SearchSortValuesAndFormats; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.MultiValueMode; import org.elasticsearch.search.builder.SearchSourceBuilder; @@ -60,7 +62,6 @@ import java.util.Objects; import java.util.function.Function; -import static org.elasticsearch.index.mapper.DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER; import static org.elasticsearch.index.mapper.DateFieldMapper.Resolution.MILLISECONDS; import static org.elasticsearch.index.mapper.DateFieldMapper.Resolution.NANOSECONDS; import static org.elasticsearch.index.search.NestedHelper.parentObject; @@ -358,10 +359,10 @@ public boolean canRewriteToMatchNone() { /** * Returns whether some values of the given {@link QueryShardContext#getIndexReader()} are within the - * primary sort value provided in the rawBottomSortValues. + * primary sort value provided in the bottomSortValues. */ - public boolean isBottomSortShardDisjoint(QueryShardContext context, Object[] rawBottomSortValues) { - if (rawBottomSortValues == null || rawBottomSortValues.length == 0) { + public boolean isBottomSortShardDisjoint(QueryShardContext context, SearchSortValuesAndFormats bottomSortValues) throws IOException { + if (bottomSortValues == null || bottomSortValues.getRawSortValues().length == 0) { return false; } @@ -376,15 +377,23 @@ public boolean isBottomSortShardDisjoint(QueryShardContext context, Object[] raw if (fieldType.indexOptions() == IndexOptions.NONE) { return false; } - Object minValue = order() == SortOrder.DESC ? rawBottomSortValues[0] : null; - Object maxValue = order() == SortOrder.DESC ? null : rawBottomSortValues[0]; - try { - MappedFieldType.Relation relation = fieldType.isFieldWithinQuery(context.getIndexReader(), minValue, maxValue, - true, true, null, DEFAULT_DATE_TIME_FORMATTER.toDateMathParser(), context); - return relation == MappedFieldType.Relation.DISJOINT; - } catch (Exception exc) { - return false; - } + DocValueFormat docValueFormat = bottomSortValues.getSortValueFormats()[0]; + final DateMathParser dateMathParser; + if (docValueFormat instanceof DocValueFormat.DateTime) { + if (fieldType instanceof DateFieldType && ((DateFieldType) fieldType).resolution() == NANOSECONDS) { + // no matter what + docValueFormat = DocValueFormat.withNanosecondResolution(docValueFormat); + } + dateMathParser = ((DocValueFormat.DateTime) docValueFormat).getDateMathParser(); + } else { + dateMathParser = null; + } + Object bottomSortValue = bottomSortValues.getFormattedSortValues()[0]; + Object minValue = order() == SortOrder.DESC ? bottomSortValue : null; + Object maxValue = order() == SortOrder.DESC ? null : bottomSortValue; + MappedFieldType.Relation relation = fieldType.isFieldWithinQuery(context.getIndexReader(), minValue, maxValue, + true, true, null, dateMathParser, context); + return relation == MappedFieldType.Relation.DISJOINT; } @Override diff --git a/server/src/test/java/org/elasticsearch/action/search/BottomSortValuesCollectorTests.java b/server/src/test/java/org/elasticsearch/action/search/BottomSortValuesCollectorTests.java new file mode 100644 index 0000000000000..21b5b547936ae --- /dev/null +++ b/server/src/test/java/org/elasticsearch/action/search/BottomSortValuesCollectorTests.java @@ -0,0 +1,230 @@ +/* + * 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.lucene.search.FieldComparator; +import org.apache.lucene.search.FieldDoc; +import org.apache.lucene.search.SortField; +import org.apache.lucene.search.TopFieldDocs; +import org.apache.lucene.search.TotalHits; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.time.DateUtils; +import org.elasticsearch.index.mapper.DateFieldMapper; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.test.ESTestCase; + +import java.time.ZoneId; +import java.util.Arrays; + +import static org.apache.lucene.search.TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO; +import static org.elasticsearch.index.mapper.DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; + +public class BottomSortValuesCollectorTests extends ESTestCase { + public void testWithStrings() { + for (boolean reverse : new boolean[] { true, false }) { + SortField[] sortFields = new SortField[] { new SortField("foo", SortField.Type.STRING_VAL, reverse) }; + DocValueFormat[] sortFormats = new DocValueFormat[] { DocValueFormat.RAW }; + BottomSortValuesCollector collector = new BottomSortValuesCollector(3, sortFields); + collector.consumeTopDocs(createTopDocs(sortFields[0], 100, + newBytesArray("foo", "goo", "hoo")), sortFormats); + collector.consumeTopDocs(createTopDocs(sortFields[0], 100, + newBytesArray("bar", "car", "zar")), sortFormats); + collector.consumeTopDocs(createTopDocs(sortFields[0], 50, + newBytesArray()), sortFormats); + collector.consumeTopDocs(createTopDocs(sortFields[0], 50, + newBytesArray("tar", "zar", "zzz")), sortFormats); + assertThat(collector.getTotalHits(), equalTo(300L)); + assertNotNull(collector.getBottomSortValues()); + assertThat(collector.getBottomSortValues().getSortValueFormats().length, equalTo(1)); + assertThat(collector.getBottomSortValues().getSortValueFormats()[0], instanceOf(DocValueFormat.RAW.getClass())); + assertThat(collector.getBottomSortValues().getRawSortValues().length, equalTo(1)); + assertThat(collector.getBottomSortValues().getFormattedSortValues().length, equalTo(1)); + if (reverse) { + assertThat(collector.getBottomSortValues().getRawSortValues()[0], equalTo(new BytesRef("tar"))); + assertThat(collector.getBottomSortValues().getFormattedSortValues()[0], equalTo("tar")); + } else { + assertThat(collector.getBottomSortValues().getRawSortValues()[0], equalTo(new BytesRef("hoo"))); + assertThat(collector.getBottomSortValues().getFormattedSortValues()[0], equalTo("hoo")); + } + } + } + + public void testWithLongs() { + for (boolean reverse : new boolean[] { true, false }) { + SortField[] sortFields = new SortField[] { new SortField("foo", SortField.Type.LONG, reverse) }; + DocValueFormat[] sortFormats = new DocValueFormat[]{ DocValueFormat.RAW }; + BottomSortValuesCollector collector = new BottomSortValuesCollector(3, sortFields); + collector.consumeTopDocs(createTopDocs(sortFields[0], 100, + newLongArray(5L, 10L, 15L)), sortFormats); + collector.consumeTopDocs(createTopDocs(sortFields[0], 100, + newLongArray(25L, 350L, 3500L)), sortFormats); + collector.consumeTopDocs(createTopDocs(sortFields[0], 50, + newLongArray(1L, 2L, 3L)), sortFormats); + collector.consumeTopDocs(createTopDocs(sortFields[0], 50, + newLongArray()), sortFormats); + assertNotNull(collector.getBottomSortValues()); + assertThat(collector.getTotalHits(), equalTo(300L)); + assertThat(collector.getBottomSortValues().getSortValueFormats().length, equalTo(1)); + assertThat(collector.getBottomSortValues().getSortValueFormats()[0], instanceOf(DocValueFormat.RAW.getClass())); + assertThat(collector.getBottomSortValues().getRawSortValues().length, equalTo(1)); + assertThat(collector.getBottomSortValues().getFormattedSortValues().length, equalTo(1)); + if (reverse) { + assertThat(collector.getBottomSortValues().getRawSortValues()[0], equalTo(25L)); + assertThat(collector.getBottomSortValues().getFormattedSortValues()[0], equalTo(25L)); + } else { + assertThat(collector.getBottomSortValues().getRawSortValues()[0], equalTo(3L)); + assertThat(collector.getBottomSortValues().getFormattedSortValues()[0], equalTo(3L)); + } + } + } + + public void testWithDoubles() { + for (boolean reverse : new boolean[] { true, false }) { + SortField[] sortFields = new SortField[]{ new SortField("foo", SortField.Type.LONG, reverse) }; + DocValueFormat[] sortFormats = new DocValueFormat[] { DocValueFormat.RAW }; + BottomSortValuesCollector collector = new BottomSortValuesCollector(3, sortFields); + collector.consumeTopDocs(createTopDocs(sortFields[0], 100, + newDoubleArray(500d, 5000d, 6755d)), sortFormats); + collector.consumeTopDocs(createTopDocs(sortFields[0], 100, + newDoubleArray(0.1d, 1.5d, 3.5d)), sortFormats); + collector.consumeTopDocs(createTopDocs(sortFields[0], 50, + newDoubleArray()), sortFormats); + collector.consumeTopDocs(createTopDocs(sortFields[0], 50, + newDoubleArray(100d, 101d, 102d)), sortFormats); + assertThat(collector.getTotalHits(), equalTo(300L)); + assertNotNull(collector.getBottomSortValues()); + assertThat(collector.getBottomSortValues().getSortValueFormats().length, equalTo(1)); + assertThat(collector.getBottomSortValues().getSortValueFormats()[0], instanceOf(DocValueFormat.RAW.getClass())); + assertThat(collector.getBottomSortValues().getRawSortValues().length, equalTo(1)); + assertThat(collector.getBottomSortValues().getFormattedSortValues().length, equalTo(1)); + if (reverse) { + assertThat(collector.getBottomSortValues().getRawSortValues()[0], equalTo(500d)); + assertThat(collector.getBottomSortValues().getFormattedSortValues()[0], equalTo(500d)); + } else { + assertThat(collector.getBottomSortValues().getRawSortValues()[0], equalTo(3.5d)); + assertThat(collector.getBottomSortValues().getFormattedSortValues()[0], equalTo(3.5d)); + } + } + } + + public void testWithDates() { + for (boolean reverse : new boolean[] { true, false }) { + SortField[] sortFields = new SortField[]{ new SortField("foo", SortField.Type.LONG, reverse) }; + DocValueFormat[] sortFormats = new DocValueFormat[] { + new DocValueFormat.DateTime(DEFAULT_DATE_TIME_FORMATTER, ZoneId.of("UTC"), DateFieldMapper.Resolution.MILLISECONDS)}; + BottomSortValuesCollector collector = new BottomSortValuesCollector(3, sortFields); + collector.consumeTopDocs(createTopDocs(sortFields[0], 100, + newDateArray("2017-06-01T12:18:20Z", "2018-04-03T15:10:27Z", "2013-06-01T13:10:20Z")), sortFormats); + collector.consumeTopDocs(createTopDocs(sortFields[0], 100, + newDateArray("2018-05-21T08:10:10Z", "2015-02-08T15:12:34Z", "2015-01-01T13:10:30Z")), sortFormats); + collector.consumeTopDocs(createTopDocs(sortFields[0], 50, + newDateArray()), sortFormats); + collector.consumeTopDocs(createTopDocs(sortFields[0], 50, + newDateArray("2019-12-30T07:34:20Z", "2017-03-01T12:10:30Z", "2015-07-09T14:00:30Z")), sortFormats); + assertThat(collector.getTotalHits(), equalTo(300L)); + assertNotNull(collector.getBottomSortValues()); + assertThat(collector.getBottomSortValues().getSortValueFormats().length, equalTo(1)); + assertThat(collector.getBottomSortValues().getSortValueFormats()[0], instanceOf(DocValueFormat.DateTime.class)); + assertThat(collector.getBottomSortValues().getRawSortValues().length, equalTo(1)); + if (reverse) { + assertThat(collector.getBottomSortValues().getRawSortValues()[0], equalTo(1436450430000L)); + assertThat(collector.getBottomSortValues().getFormattedSortValues()[0], equalTo("2015-07-09T14:00:30.000Z")); + } else { + assertThat(collector.getBottomSortValues().getRawSortValues()[0], equalTo(1522768227000L)); + assertThat(collector.getBottomSortValues().getFormattedSortValues()[0], equalTo("2018-04-03T15:10:27.000Z")); + } + } + } + + public void testWithDateNanos() { + for (boolean reverse : new boolean[] { true, false }) { + SortField[] sortFields = new SortField[]{ new SortField("foo", SortField.Type.LONG, reverse) }; + DocValueFormat[] sortFormats = new DocValueFormat[] { + new DocValueFormat.DateTime(DEFAULT_DATE_TIME_FORMATTER, ZoneId.of("UTC"), DateFieldMapper.Resolution.NANOSECONDS)}; + BottomSortValuesCollector collector = new BottomSortValuesCollector(3, sortFields); + collector.consumeTopDocs(createTopDocs(sortFields[0], 100, + newDateNanoArray("2017-06-01T12:18:20Z", "2018-04-03T15:10:27Z", "2013-06-01T13:10:20Z")), sortFormats); + collector.consumeTopDocs(createTopDocs(sortFields[0], 100, + newDateNanoArray("2018-05-21T08:10:10Z", "2015-02-08T15:12:34Z", "2015-01-01T13:10:30Z")), sortFormats); + collector.consumeTopDocs(createTopDocs(sortFields[0], 50, + newDateNanoArray()), sortFormats); + collector.consumeTopDocs(createTopDocs(sortFields[0], 50, + newDateNanoArray("2019-12-30T07:34:20Z", "2017-03-01T12:10:30Z", "2015-07-09T14:00:30Z")), sortFormats); + assertThat(collector.getTotalHits(), equalTo(300L)); + assertNotNull(collector.getBottomSortValues()); + assertThat(collector.getBottomSortValues().getSortValueFormats().length, equalTo(1)); + assertThat(collector.getBottomSortValues().getSortValueFormats()[0], instanceOf(DocValueFormat.DateTime.class)); + assertThat(collector.getBottomSortValues().getRawSortValues().length, equalTo(1)); + if (reverse) { + assertThat(collector.getBottomSortValues().getRawSortValues()[0], equalTo(1436450430000000000L)); + assertThat(collector.getBottomSortValues().getFormattedSortValues()[0], equalTo("2015-07-09T14:00:30.000Z")); + } else { + assertThat(collector.getBottomSortValues().getRawSortValues()[0], equalTo(1522768227000000000L)); + assertThat(collector.getBottomSortValues().getFormattedSortValues()[0], equalTo("2018-04-03T15:10:27.000Z")); + } + } + } + + private Object[] newDoubleArray(Double... values) { + return values; + } + + private Object[] newLongArray(Long... values) { + return values; + } + + private Object[] newBytesArray(String... values) { + BytesRef[] bytesRefs = new BytesRef[values.length]; + for (int i = 0; i < bytesRefs.length; i++) { + bytesRefs[i] = new BytesRef(values[i]); + } + return bytesRefs; + } + + private Object[] newDateArray(String... values) { + Long[] longs = new Long[values.length]; + for (int i = 0; i < values.length; i++) { + longs[i] = DEFAULT_DATE_TIME_FORMATTER.parseMillis(values[i]); + } + return longs; + } + + private Object[] newDateNanoArray(String... values) { + Long[] longs = new Long[values.length]; + for (int i = 0; i < values.length; i++) { + longs[i] = DateUtils.toNanoSeconds(DEFAULT_DATE_TIME_FORMATTER.parseMillis(values[i])); + } + return longs; + } + + private TopFieldDocs createTopDocs(SortField sortField, int totalHits, Object[] values) { + FieldDoc[] fieldDocs = new FieldDoc[values.length]; + FieldComparator cmp = sortField.getComparator(1, 0); + for (int i = 0; i < values.length; i++) { + fieldDocs[i] = new FieldDoc(i, Float.NaN, new Object[] { values[i] }); + } + int reverseMul = sortField.getReverse() ? -1 : 1; + Arrays.sort(fieldDocs, (o1, o2) -> reverseMul * cmp.compareValues(o1.fields[0], o2.fields[0])); + return new TopFieldDocs(new TotalHits(totalHits, GREATER_THAN_OR_EQUAL_TO), + fieldDocs, new SortField[] { sortField }); + } +} diff --git a/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java b/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java index 297f83015bac8..edc409deb2d49 100644 --- a/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java @@ -79,8 +79,8 @@ public void sendExecuteQuery(Transport.Connection connection, ShardSearchRequest if (request.canReturnNullResponseIfMatchNoDocs()) { canReturnNullResponse.set(true); } - if (request.getRawBottomSortValues() != null) { - assertNotEquals(shardId, (int) request.getRawBottomSortValues()[0]); + if (request.getBottomSortValues() != null) { + assertNotEquals(shardId, (int) request.getBottomSortValues().getFormattedSortValues()[0]); numWithTopDocs.incrementAndGet(); } QuerySearchResult queryResult = new QuerySearchResult(123, diff --git a/server/src/test/java/org/elasticsearch/search/SearchSortValuesAndFormatsTests.java b/server/src/test/java/org/elasticsearch/search/SearchSortValuesAndFormatsTests.java new file mode 100644 index 0000000000000..7ef7218ca1997 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/search/SearchSortValuesAndFormatsTests.java @@ -0,0 +1,80 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search; + +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.lucene.LuceneTests; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.test.AbstractWireSerializingTestCase; +import org.junit.Before; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +public class SearchSortValuesAndFormatsTests extends AbstractWireSerializingTestCase { + private NamedWriteableRegistry namedWriteableRegistry; + + @Before + public void initRegistry() { + SearchModule searchModule = new SearchModule(Settings.EMPTY, Collections.emptyList()); + List entries = new ArrayList<>(); + entries.addAll(searchModule.getNamedWriteables()); + namedWriteableRegistry = new NamedWriteableRegistry(entries); + } + + @Override + protected NamedWriteableRegistry getNamedWriteableRegistry() { + return namedWriteableRegistry; + } + + @Override + protected Writeable.Reader instanceReader() { + return SearchSortValuesAndFormats::new; + } + + @Override + protected SearchSortValuesAndFormats createTestInstance() { + return randomInstance(); + } + + @Override + protected SearchSortValuesAndFormats mutateInstance(SearchSortValuesAndFormats instance) { + Object[] sortValues = instance.getFormattedSortValues(); + Object[] newValues = Arrays.copyOf(sortValues, sortValues.length + 1); + DocValueFormat[] newFormats = Arrays.copyOf(instance.getSortValueFormats(), sortValues.length + 1); + newValues[sortValues.length] = LuceneTests.randomSortValue(); + newFormats[sortValues.length] = DocValueFormat.RAW; + return new SearchSortValuesAndFormats(newValues, newFormats); + } + + public static SearchSortValuesAndFormats randomInstance() { + int size = randomIntBetween(1, 20); + Object[] values = new Object[size]; + DocValueFormat[] sortValueFormats = new DocValueFormat[size]; + for (int i = 0; i < size; i++) { + values[i] = LuceneTests.randomSortValue(); + sortValueFormats[i] = DocValueFormat.RAW; + } + return new SearchSortValuesAndFormats(values, sortValueFormats); + } +} diff --git a/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java b/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java index ca0eeb2bb46c2..a9759c6298e83 100644 --- a/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java +++ b/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java @@ -40,6 +40,7 @@ import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.InvalidAliasNameException; import org.elasticsearch.search.AbstractSearchTestCase; +import org.elasticsearch.search.SearchSortValuesAndFormatsTests; import java.io.IOException; @@ -95,9 +96,7 @@ private ShardSearchRequest createShardSearchRequest() throws IOException { Math.abs(randomLong()), randomAlphaOfLengthBetween(3, 10), routings); req.canReturnNullResponseIfMatchNoDocs(randomBoolean()); if (randomBoolean()) { - req.setRawBottomSortValues( - randomArray(randomIntBetween(1, 3), Object[]::new, - () -> randomBoolean() ? randomUnicodeOfLength(10) : randomLong())); + req.setBottomSortValues(SearchSortValuesAndFormatsTests.randomInstance()); } return req; } diff --git a/server/src/test/java/org/elasticsearch/search/sort/FieldSortBuilderTests.java b/server/src/test/java/org/elasticsearch/search/sort/FieldSortBuilderTests.java index 6fe340d67471b..b457e560c65d4 100644 --- a/server/src/test/java/org/elasticsearch/search/sort/FieldSortBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/search/sort/FieldSortBuilderTests.java @@ -62,6 +62,7 @@ import org.elasticsearch.index.query.RangeQueryBuilder; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.MultiValueMode; +import org.elasticsearch.search.SearchSortValuesAndFormats; import org.elasticsearch.search.builder.SearchSourceBuilder; import java.io.IOException; @@ -578,7 +579,8 @@ public void testIsBottomSortShardDisjoint() throws Exception { FieldSortBuilder fieldSort = SortBuilders.fieldSort("custom-date"); try (DirectoryReader reader = writer.getReader()) { QueryShardContext context = createMockShardContext(new IndexSearcher(reader)); - assertTrue(fieldSort.isBottomSortShardDisjoint(context, new Object[] { 0L })); + assertTrue(fieldSort.isBottomSortShardDisjoint(context, + new SearchSortValuesAndFormats(new Object[] { 0L }, new DocValueFormat[] { DocValueFormat.RAW }))); } for (int i = 0; i < numDocs; i++) { Document doc = new Document(); @@ -592,18 +594,26 @@ public void testIsBottomSortShardDisjoint() throws Exception { try (DirectoryReader reader = writer.getReader()) { QueryShardContext context = createMockShardContext(new IndexSearcher(reader)); assertFalse(fieldSort.isBottomSortShardDisjoint(context, null)); - assertFalse(fieldSort.isBottomSortShardDisjoint(context, new Object[] { minValue })); - assertTrue(fieldSort.isBottomSortShardDisjoint(context, new Object[] { minValue-1 })); - assertFalse(fieldSort.isBottomSortShardDisjoint(context, new Object[] { minValue+1 })); + assertFalse(fieldSort.isBottomSortShardDisjoint(context, + new SearchSortValuesAndFormats(new Object[] { minValue }, new DocValueFormat[] { DocValueFormat.RAW }))); + assertTrue(fieldSort.isBottomSortShardDisjoint(context, + new SearchSortValuesAndFormats(new Object[] { minValue-1 }, new DocValueFormat[] { DocValueFormat.RAW }))); + assertFalse(fieldSort.isBottomSortShardDisjoint(context, + new SearchSortValuesAndFormats(new Object[] { minValue+1 }, new DocValueFormat[] { DocValueFormat.RAW }))); fieldSort.order(SortOrder.DESC); - assertTrue(fieldSort.isBottomSortShardDisjoint(context, new Object[] { maxValue+1 })); - assertFalse(fieldSort.isBottomSortShardDisjoint(context, new Object[] { maxValue })); - assertFalse(fieldSort.isBottomSortShardDisjoint(context, new Object[] { minValue })); + assertTrue(fieldSort.isBottomSortShardDisjoint(context, + new SearchSortValuesAndFormats(new Object[] { maxValue+1 }, new DocValueFormat[] { DocValueFormat.RAW }))); + assertFalse(fieldSort.isBottomSortShardDisjoint(context, + new SearchSortValuesAndFormats(new Object[] { maxValue }, new DocValueFormat[] { DocValueFormat.RAW }))); + assertFalse(fieldSort.isBottomSortShardDisjoint(context, + new SearchSortValuesAndFormats(new Object[] { minValue }, new DocValueFormat[] { DocValueFormat.RAW }))); fieldSort.setNestedSort(new NestedSortBuilder("empty")); - assertFalse(fieldSort.isBottomSortShardDisjoint(context, new Object[] { minValue-1 })); + assertFalse(fieldSort.isBottomSortShardDisjoint(context, + new SearchSortValuesAndFormats(new Object[] { minValue-1 }, new DocValueFormat[] { DocValueFormat.RAW }))); fieldSort.setNestedSort(null); fieldSort.missing("100"); - assertFalse(fieldSort.isBottomSortShardDisjoint(context, new Object[] { maxValue+1 })); + assertFalse(fieldSort.isBottomSortShardDisjoint(context, + new SearchSortValuesAndFormats(new Object[] { maxValue+1 }, new DocValueFormat[] { DocValueFormat.RAW }))); } } } From 6489ffeefff1e0ceb43be2e2a16384b1a96567ed Mon Sep 17 00:00:00 2001 From: jimczi Date: Mon, 9 Mar 2020 20:43:08 +0100 Subject: [PATCH 29/37] fix unset of track total hits in distributed search --- .../action/search/SearchQueryThenFetchAsyncAction.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 84950bcad1a5d..5a7d5648d0ca6 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -113,7 +113,7 @@ private ShardSearchRequest rewriteShardSearchRequest(ShardSearchRequest request) // disable tracking total hits if we already reached the required estimation. if (trackTotalHitsUpTo != SearchContext.TRACK_TOTAL_HITS_ACCURATE - && bottomSortCollector.getTotalHits() >= trackTotalHitsUpTo) { + && bottomSortCollector.getTotalHits() > trackTotalHitsUpTo) { request.source(request.source().shallowCopy().trackTotalHits(false)); } From 73aa68159154be7d8da13f34e15d8a4c51c9ec60 Mon Sep 17 00:00:00 2001 From: jimczi Date: Tue, 10 Mar 2020 01:07:34 +0100 Subject: [PATCH 30/37] fix compilation test after merging with master --- .../action/search/AbstractSearchAsyncAction.java | 2 +- .../action/search/SearchQueryThenFetchAsyncActionTests.java | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index 8c1cd645ff00f..b1d27680f5b28 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -77,7 +77,7 @@ abstract class AbstractSearchAsyncAction exten **/ private final BiFunction nodeIdToConnection; private final SearchTask task; - private final SearchPhaseResults results; + final SearchPhaseResults results; private final ClusterState clusterState; private final Map aliasFilter; private final Map concreteIndexBoosts; diff --git a/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java b/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java index edc409deb2d49..38b5f6e21ca79 100644 --- a/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java @@ -38,6 +38,7 @@ import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.internal.AliasFilter; +import org.elasticsearch.search.internal.SearchContextId; import org.elasticsearch.search.internal.ShardSearchRequest; import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.search.sort.SortBuilders; @@ -83,7 +84,7 @@ public void sendExecuteQuery(Transport.Connection connection, ShardSearchRequest assertNotEquals(shardId, (int) request.getBottomSortValues().getFormattedSortValues()[0]); numWithTopDocs.incrementAndGet(); } - QuerySearchResult queryResult = new QuerySearchResult(123, + QuerySearchResult queryResult = new QuerySearchResult(new SearchContextId("N/A", 123), new SearchShardTarget("node1", new ShardId("idx", "na", shardId), null, OriginalIndices.NONE)); SortField sortField = new SortField("timestamp", SortField.Type.LONG); queryResult.topDocs(new TopDocsAndMaxScore(new TopFieldDocs( @@ -117,7 +118,7 @@ public void sendExecuteQuery(Transport.Connection connection, ShardSearchRequest searchTransportService, (clusterAlias, node) -> lookup.get(node), Collections.singletonMap("_na_", new AliasFilter(null, Strings.EMPTY_ARRAY)), Collections.emptyMap(), Collections.emptyMap(), controller, EsExecutors.newDirectExecutorService(), searchRequest, - null, shardsIter, timeProvider, 0, task, + null, shardsIter, timeProvider, null, task, SearchResponse.Clusters.EMPTY) { @Override protected SearchPhase getNextPhase(SearchPhaseResults results, SearchPhaseContext context) { @@ -142,6 +143,5 @@ public void run() { assertThat(phase.sortedTopDocs.scoreDocs[0], instanceOf(FieldDoc.class)); assertThat(((FieldDoc) phase.sortedTopDocs.scoreDocs[0]).fields.length, equalTo(1)); assertThat(((FieldDoc) phase.sortedTopDocs.scoreDocs[0]).fields[0], equalTo(0)); - } } From 36bec996a78aa340dc7686cbdd0f8af3d57c4263 Mon Sep 17 00:00:00 2001 From: jimczi Date: Tue, 10 Mar 2020 10:48:12 +0100 Subject: [PATCH 31/37] fix partial comment --- .../java/org/elasticsearch/search/sort/FieldSortBuilder.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java b/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java index c892703272e5b..17ba31bf8954d 100644 --- a/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java @@ -381,7 +381,8 @@ public boolean isBottomSortShardDisjoint(QueryShardContext context, SearchSortVa final DateMathParser dateMathParser; if (docValueFormat instanceof DocValueFormat.DateTime) { if (fieldType instanceof DateFieldType && ((DateFieldType) fieldType).resolution() == NANOSECONDS) { - // no matter what + // we parse the formatted value with the resolution of the local field because + // the provided format can use a different one (date vs date_nanos). docValueFormat = DocValueFormat.withNanosecondResolution(docValueFormat); } dateMathParser = ((DocValueFormat.DateTime) docValueFormat).getDateMathParser(); From 7a5422282569cd68acbf04268939247e765513d3 Mon Sep 17 00:00:00 2001 From: jimczi Date: Wed, 11 Mar 2020 15:00:18 +0100 Subject: [PATCH 32/37] iter --- .../search/BottomSortValuesCollector.java | 23 +++++++++++ .../search/sort/FieldSortBuilder.java | 12 ++++-- .../BottomSortValuesCollectorTests.java | 41 +++++++++++++++++-- 3 files changed, 69 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/BottomSortValuesCollector.java b/server/src/main/java/org/elasticsearch/action/search/BottomSortValuesCollector.java index 052c927e2e45a..06df5cd6123b7 100644 --- a/server/src/main/java/org/elasticsearch/action/search/BottomSortValuesCollector.java +++ b/server/src/main/java/org/elasticsearch/action/search/BottomSortValuesCollector.java @@ -31,6 +31,7 @@ */ class BottomSortValuesCollector { private final int topNSize; + private final SortField[] sortFields; private final FieldComparator[] comparators; private final int[] reverseMuls; @@ -41,6 +42,7 @@ class BottomSortValuesCollector { this.topNSize = topNSize; this.comparators = new FieldComparator[sortFields.length]; this.reverseMuls = new int[sortFields.length]; + this.sortFields = sortFields; for (int i = 0; i < sortFields.length; i++) { comparators[i] = sortFields[i].getComparator(1, i); reverseMuls[i] = sortFields[i].getReverse() ? -1 : 1; @@ -60,6 +62,9 @@ SearchSortValuesAndFormats getBottomSortValues() { synchronized void consumeTopDocs(TopFieldDocs topDocs, DocValueFormat[] sortValuesFormat) { totalHits += topDocs.totalHits.value; + if (validateShardSortFields(topDocs.fields) == false) { + return; + } FieldDoc shardBottomDoc = extractBottom(topDocs); if (shardBottomDoc == null) { @@ -71,6 +76,24 @@ synchronized void consumeTopDocs(TopFieldDocs topDocs, DocValueFormat[] sortValu } } + /** + * @return false if the provided {@link SortField} array differs + * from the initial {@link BottomSortValuesCollector#sortFields}. + */ + private boolean validateShardSortFields(SortField[] shardSortFields) { + for (int i = 0; i < shardSortFields.length; i++) { + if (shardSortFields[i].equals(sortFields[i]) == false) { + // ignore shards response that would make the sort incompatible + // (e.g.: mixing keyword/numeric or long/double). + // TODO: we should fail the entire request because the topdocs + // merge will likely fail later but this is not possible with + // the current async logic that only allows shard failures here. + return false; + } + } + return true; + } + private FieldDoc extractBottom(TopFieldDocs topDocs) { return topNSize > 0 && topDocs.scoreDocs.length == topNSize ? (FieldDoc) topDocs.scoreDocs[topNSize-1] : null; diff --git a/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java b/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java index 17ba31bf8954d..044c59d24a809 100644 --- a/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java @@ -26,6 +26,7 @@ import org.apache.lucene.index.PointValues; import org.apache.lucene.index.Terms; import org.apache.lucene.search.SortField; +import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.Version; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; @@ -392,9 +393,14 @@ public boolean isBottomSortShardDisjoint(QueryShardContext context, SearchSortVa Object bottomSortValue = bottomSortValues.getFormattedSortValues()[0]; Object minValue = order() == SortOrder.DESC ? bottomSortValue : null; Object maxValue = order() == SortOrder.DESC ? null : bottomSortValue; - MappedFieldType.Relation relation = fieldType.isFieldWithinQuery(context.getIndexReader(), minValue, maxValue, - true, true, null, dateMathParser, context); - return relation == MappedFieldType.Relation.DISJOINT; + try { + MappedFieldType.Relation relation = fieldType.isFieldWithinQuery(context.getIndexReader(), minValue, maxValue, + true, true, null, dateMathParser, context); + return relation == MappedFieldType.Relation.DISJOINT; + } catch (ElasticsearchParseException exc) { + // can happen if the sort field is mapped differently in another search index + return false; + } } @Override diff --git a/server/src/test/java/org/elasticsearch/action/search/BottomSortValuesCollectorTests.java b/server/src/test/java/org/elasticsearch/action/search/BottomSortValuesCollectorTests.java index 21b5b547936ae..7c626f33e9c40 100644 --- a/server/src/test/java/org/elasticsearch/action/search/BottomSortValuesCollectorTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/BottomSortValuesCollectorTests.java @@ -52,7 +52,9 @@ public void testWithStrings() { newBytesArray()), sortFormats); collector.consumeTopDocs(createTopDocs(sortFields[0], 50, newBytesArray("tar", "zar", "zzz")), sortFormats); - assertThat(collector.getTotalHits(), equalTo(300L)); + collector.consumeTopDocs(createTopDocs(sortFields[0], 50, + newBytesArray(null, null, "zzz")), sortFormats); + assertThat(collector.getTotalHits(), equalTo(350L)); assertNotNull(collector.getBottomSortValues()); assertThat(collector.getBottomSortValues().getSortValueFormats().length, equalTo(1)); assertThat(collector.getBottomSortValues().getSortValueFormats()[0], instanceOf(DocValueFormat.RAW.getClass())); @@ -81,8 +83,11 @@ public void testWithLongs() { newLongArray(1L, 2L, 3L)), sortFormats); collector.consumeTopDocs(createTopDocs(sortFields[0], 50, newLongArray()), sortFormats); + // ignore bottom if we have less top docs than the requested size + collector.consumeTopDocs(createTopDocs(sortFields[0], 1, + newLongArray(-100L)), sortFormats); assertNotNull(collector.getBottomSortValues()); - assertThat(collector.getTotalHits(), equalTo(300L)); + assertThat(collector.getTotalHits(), equalTo(301L)); assertThat(collector.getBottomSortValues().getSortValueFormats().length, equalTo(1)); assertThat(collector.getBottomSortValues().getSortValueFormats()[0], instanceOf(DocValueFormat.RAW.getClass())); assertThat(collector.getBottomSortValues().getRawSortValues().length, equalTo(1)); @@ -110,7 +115,10 @@ public void testWithDoubles() { newDoubleArray()), sortFormats); collector.consumeTopDocs(createTopDocs(sortFields[0], 50, newDoubleArray(100d, 101d, 102d)), sortFormats); - assertThat(collector.getTotalHits(), equalTo(300L)); + // ignore bottom if we have less top docs than the requested size + collector.consumeTopDocs(createTopDocs(sortFields[0], 2, + newDoubleArray(0d, 1d)), sortFormats); + assertThat(collector.getTotalHits(), equalTo(302L)); assertNotNull(collector.getBottomSortValues()); assertThat(collector.getBottomSortValues().getSortValueFormats().length, equalTo(1)); assertThat(collector.getBottomSortValues().getSortValueFormats()[0], instanceOf(DocValueFormat.RAW.getClass())); @@ -184,6 +192,31 @@ public void testWithDateNanos() { } } + public void testWithMixedTypes() { + for (boolean reverse : new boolean[] { true, false }) { + SortField[] sortFields = new SortField[] { new SortField("foo", SortField.Type.LONG, reverse) }; + SortField[] otherSortFields = new SortField[] { new SortField("foo", SortField.Type.STRING_VAL, reverse) }; + DocValueFormat[] sortFormats = new DocValueFormat[] { DocValueFormat.RAW }; + BottomSortValuesCollector collector = new BottomSortValuesCollector(3, sortFields); + collector.consumeTopDocs(createTopDocs(sortFields[0], 100, + newLongArray(1000L, 100L, 10L)), sortFormats); + collector.consumeTopDocs(createTopDocs(otherSortFields[0], 50, + newBytesArray("foo", "bar", "zoo")), sortFormats); + assertThat(collector.getTotalHits(), equalTo(150L)); + assertNotNull(collector.getBottomSortValues()); + assertThat(collector.getBottomSortValues().getSortValueFormats().length, equalTo(1)); + assertThat(collector.getBottomSortValues().getSortValueFormats()[0], instanceOf(DocValueFormat.RAW.getClass())); + assertThat(collector.getBottomSortValues().getRawSortValues().length, equalTo(1)); + if (reverse) { + assertThat(collector.getBottomSortValues().getRawSortValues()[0], equalTo(10L)); + assertThat(collector.getBottomSortValues().getFormattedSortValues()[0], equalTo(10L)); + } else { + assertThat(collector.getBottomSortValues().getRawSortValues()[0], equalTo(1000L)); + assertThat(collector.getBottomSortValues().getFormattedSortValues()[0], equalTo(1000L)); + } + } + } + private Object[] newDoubleArray(Double... values) { return values; } @@ -195,7 +228,7 @@ private Object[] newLongArray(Long... values) { private Object[] newBytesArray(String... values) { BytesRef[] bytesRefs = new BytesRef[values.length]; for (int i = 0; i < bytesRefs.length; i++) { - bytesRefs[i] = new BytesRef(values[i]); + bytesRefs[i] = values[i] == null ? null : new BytesRef(values[i]); } return bytesRefs; } From abda5305ee9602bb9a4c3b9c733c57b41b2fb268 Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Fri, 13 Mar 2020 12:22:48 +0100 Subject: [PATCH 33/37] Add stricter assertions Co-Authored-By: Adrien Grand --- .../org/elasticsearch/search/SearchSortValuesAndFormats.java | 1 + 1 file changed, 1 insertion(+) diff --git a/server/src/main/java/org/elasticsearch/search/SearchSortValuesAndFormats.java b/server/src/main/java/org/elasticsearch/search/SearchSortValuesAndFormats.java index 5127c6e56d669..528a796c004bf 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchSortValuesAndFormats.java +++ b/server/src/main/java/org/elasticsearch/search/SearchSortValuesAndFormats.java @@ -47,6 +47,7 @@ public SearchSortValuesAndFormats(Object[] rawSortValues, DocValueFormat[] sortV } else if (sortValue instanceof Double) { this.formattedSortValues[i] = sortValueFormats[i].format((double) sortValue); } else { + assert sortValue == null : "Sort values must be a BytesRef, Long or Double, but got " + sortValue.getClass() + ": " + sortValue; this.formattedSortValues[i] = sortValue; } } From 06cbc1981b63cca2b8ab23001f0e14523fd21c51 Mon Sep 17 00:00:00 2001 From: jimczi Date: Fri, 13 Mar 2020 12:37:45 +0100 Subject: [PATCH 34/37] checkstyle --- .../org/elasticsearch/search/SearchSortValuesAndFormats.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/search/SearchSortValuesAndFormats.java b/server/src/main/java/org/elasticsearch/search/SearchSortValuesAndFormats.java index 528a796c004bf..75c2b707f00c2 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchSortValuesAndFormats.java +++ b/server/src/main/java/org/elasticsearch/search/SearchSortValuesAndFormats.java @@ -47,7 +47,8 @@ public SearchSortValuesAndFormats(Object[] rawSortValues, DocValueFormat[] sortV } else if (sortValue instanceof Double) { this.formattedSortValues[i] = sortValueFormats[i].format((double) sortValue); } else { - assert sortValue == null : "Sort values must be a BytesRef, Long or Double, but got " + sortValue.getClass() + ": " + sortValue; + assert sortValue == null : "Sort values must be a BytesRef, Long or Double, but got " + + sortValue.getClass() + ": " + sortValue; this.formattedSortValues[i] = sortValue; } } From 14542f00db2cca904381678a3a26a20ee77e229b Mon Sep 17 00:00:00 2001 From: jimczi Date: Mon, 16 Mar 2020 09:26:14 +0100 Subject: [PATCH 35/37] sort values can be integers and floats (_doc and _score) --- .../org/elasticsearch/search/SearchSortValuesAndFormats.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/search/SearchSortValuesAndFormats.java b/server/src/main/java/org/elasticsearch/search/SearchSortValuesAndFormats.java index 75c2b707f00c2..f5f80836cde8f 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchSortValuesAndFormats.java +++ b/server/src/main/java/org/elasticsearch/search/SearchSortValuesAndFormats.java @@ -46,8 +46,11 @@ public SearchSortValuesAndFormats(Object[] rawSortValues, DocValueFormat[] sortV this.formattedSortValues[i] = sortValueFormats[i].format((long) sortValue); } else if (sortValue instanceof Double) { this.formattedSortValues[i] = sortValueFormats[i].format((double) sortValue); + } else if (sortValue instanceof Float || sortValue instanceof Integer) { + // sort by _score or _doc + this.formattedSortValues[i] = sortValue; } else { - assert sortValue == null : "Sort values must be a BytesRef, Long or Double, but got " + assert sortValue == null : "Sort values must be a BytesRef, Long, Integer, Double or Float, but got " + sortValue.getClass() + ": " + sortValue; this.formattedSortValues[i] = sortValue; } From 8ad41c26ecd936a74229795cc9edbff17f6057ab Mon Sep 17 00:00:00 2001 From: jimczi Date: Mon, 16 Mar 2020 12:07:29 +0100 Subject: [PATCH 36/37] fix test to handle accepted raw sort format --- .../SearchSortValuesAndFormatsTests.java | 27 ++++++++++++++++--- 1 file changed, 23 insertions(+), 4 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/search/SearchSortValuesAndFormatsTests.java b/server/src/test/java/org/elasticsearch/search/SearchSortValuesAndFormatsTests.java index 7ef7218ca1997..2273ab3af0db7 100644 --- a/server/src/test/java/org/elasticsearch/search/SearchSortValuesAndFormatsTests.java +++ b/server/src/test/java/org/elasticsearch/search/SearchSortValuesAndFormatsTests.java @@ -19,9 +19,9 @@ package org.elasticsearch.search; +import org.apache.lucene.util.BytesRef; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.Writeable; -import org.elasticsearch.common.lucene.LuceneTests; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.test.AbstractWireSerializingTestCase; import org.junit.Before; @@ -59,20 +59,39 @@ protected SearchSortValuesAndFormats createTestInstance() { @Override protected SearchSortValuesAndFormats mutateInstance(SearchSortValuesAndFormats instance) { - Object[] sortValues = instance.getFormattedSortValues(); + Object[] sortValues = instance.getRawSortValues(); Object[] newValues = Arrays.copyOf(sortValues, sortValues.length + 1); DocValueFormat[] newFormats = Arrays.copyOf(instance.getSortValueFormats(), sortValues.length + 1); - newValues[sortValues.length] = LuceneTests.randomSortValue(); + newValues[sortValues.length] = randomSortValue(); newFormats[sortValues.length] = DocValueFormat.RAW; return new SearchSortValuesAndFormats(newValues, newFormats); } + private static Object randomSortValue() { + switch(randomIntBetween(0, 5)) { + case 0: + return null; + case 1: + return new BytesRef(randomAlphaOfLengthBetween(3, 10)); + case 2: + return randomInt(); + case 3: + return randomLong(); + case 4: + return randomFloat(); + case 5: + return randomDouble(); + default: + throw new UnsupportedOperationException(); + } + } + public static SearchSortValuesAndFormats randomInstance() { int size = randomIntBetween(1, 20); Object[] values = new Object[size]; DocValueFormat[] sortValueFormats = new DocValueFormat[size]; for (int i = 0; i < size; i++) { - values[i] = LuceneTests.randomSortValue(); + values[i] = randomSortValue(); sortValueFormats[i] = DocValueFormat.RAW; } return new SearchSortValuesAndFormats(values, sortValueFormats); From e4e8b0224b70f1423f156b3081de5d5491b4b43a Mon Sep 17 00:00:00 2001 From: jimczi Date: Tue, 17 Mar 2020 09:38:43 +0100 Subject: [PATCH 37/37] fix compil after master merge --- .../SearchQueryThenFetchAsyncActionTests.java | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java b/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java index 38b5f6e21ca79..0d4e0ede61167 100644 --- a/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java @@ -36,6 +36,7 @@ import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.SearchContextId; @@ -111,8 +112,17 @@ public void sendExecuteQuery(Transport.Connection connection, ShardSearchRequest .trackTotalHitsUpTo(2) .sort(SortBuilders.fieldSort("timestamp"))); searchRequest.allowPartialSearchResults(false); - SearchPhaseController controller = new SearchPhaseController((b) -> - new InternalAggregation.ReduceContext(BigArrays.NON_RECYCLING_INSTANCE, null, b)); + SearchPhaseController controller = new SearchPhaseController((b) -> new InternalAggregation.ReduceContextBuilder() { + @Override + public InternalAggregation.ReduceContext forPartialReduction() { + return InternalAggregation.ReduceContext.forPartialReduction(BigArrays.NON_RECYCLING_INSTANCE, null); + } + + public InternalAggregation.ReduceContext forFinalReduction() { + return InternalAggregation.ReduceContext.forFinalReduction( + BigArrays.NON_RECYCLING_INSTANCE, null, b -> {}, PipelineAggregator.PipelineTree.EMPTY); + }; + }); SearchTask task = new SearchTask(0, "n/a", "n/a", "test", null, Collections.emptyMap()); SearchQueryThenFetchAsyncAction action = new SearchQueryThenFetchAsyncAction(logger, searchTransportService, (clusterAlias, node) -> lookup.get(node),