forked from opensearch-project/OpenSearch
-
Notifications
You must be signed in to change notification settings - Fork 0
/
QueryPhase.java
451 lines (415 loc) · 20.4 KB
/
QueryPhase.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
/*
* 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.
*/
/*
* Modifications Copyright OpenSearch Contributors. See
* GitHub history for details.
*/
package org.opensearch.search.query;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReaderContext;
import org.opensearch.common.unit.TimeValue;
import org.opensearch.lucene.queries.MinDocQuery;
import org.opensearch.lucene.queries.SearchAfterSortedDocQuery;
import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.ConstantScoreQuery;
import org.apache.lucene.search.FieldDoc;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.search.TotalHits;
import org.opensearch.action.search.SearchShardTask;
import org.opensearch.common.Booleans;
import org.opensearch.common.lucene.Lucene;
import org.opensearch.common.lucene.search.TopDocsAndMaxScore;
import org.opensearch.common.util.concurrent.EWMATrackingThreadPoolExecutor;
import org.opensearch.search.DocValueFormat;
import org.opensearch.search.SearchContextSourcePrinter;
import org.opensearch.search.SearchService;
import org.opensearch.search.aggregations.AggregationPhase;
import org.opensearch.search.internal.ContextIndexSearcher;
import org.opensearch.search.internal.ScrollContext;
import org.opensearch.search.internal.SearchContext;
import org.opensearch.search.profile.ProfileShardResult;
import org.opensearch.search.profile.SearchProfileShardResults;
import org.opensearch.search.profile.query.InternalProfileCollector;
import org.opensearch.search.rescore.RescorePhase;
import org.opensearch.search.sort.SortAndFormats;
import org.opensearch.search.suggest.SuggestPhase;
import org.opensearch.tasks.TaskCancelledException;
import org.opensearch.threadpool.ThreadPool;
import java.io.IOException;
import java.util.LinkedList;
import java.util.Objects;
import java.util.concurrent.ExecutorService;
import java.util.function.LongSupplier;
import static org.opensearch.search.query.QueryCollectorContext.createEarlyTerminationCollectorContext;
import static org.opensearch.search.query.QueryCollectorContext.createFilteredCollectorContext;
import static org.opensearch.search.query.QueryCollectorContext.createMinScoreCollectorContext;
import static org.opensearch.search.query.QueryCollectorContext.createMultiCollectorContext;
import static org.opensearch.search.query.TopDocsCollectorContext.createTopDocsCollectorContext;
/**
* Query phase of a search request, used to run the query and get back from each shard information about the matching documents
* (document ids and score or sort criteria) so that matches can be reduced on the coordinating node
*
* @opensearch.internal
*/
public class QueryPhase {
private static final Logger LOGGER = LogManager.getLogger(QueryPhase.class);
// TODO: remove this property
public static final boolean SYS_PROP_REWRITE_SORT = Booleans.parseBoolean(System.getProperty("opensearch.search.rewrite_sort", "true"));
public static final QueryPhaseSearcher DEFAULT_QUERY_PHASE_SEARCHER = new DefaultQueryPhaseSearcher();
private final QueryPhaseSearcher queryPhaseSearcher;
private final AggregationPhase aggregationPhase;
private final SuggestPhase suggestPhase;
private final RescorePhase rescorePhase;
public QueryPhase() {
this(DEFAULT_QUERY_PHASE_SEARCHER);
}
public QueryPhase(QueryPhaseSearcher queryPhaseSearcher) {
this.queryPhaseSearcher = Objects.requireNonNull(queryPhaseSearcher, "QueryPhaseSearcher is required");
this.aggregationPhase = new AggregationPhase();
this.suggestPhase = new SuggestPhase();
this.rescorePhase = new RescorePhase();
}
public void preProcess(SearchContext context) {
final Runnable cancellation;
if (context.lowLevelCancellation()) {
cancellation = context.searcher().addQueryCancellation(() -> {
SearchShardTask task = context.getTask();
if (task != null && task.isCancelled()) {
throw new TaskCancelledException("cancelled task with reason: " + task.getReasonCancelled());
}
});
} else {
cancellation = null;
}
try {
context.preProcess(true);
} finally {
if (cancellation != null) {
context.searcher().removeQueryCancellation(cancellation);
}
}
}
public void execute(SearchContext searchContext) throws QueryPhaseExecutionException {
if (searchContext.hasOnlySuggest()) {
suggestPhase.execute(searchContext);
searchContext.queryResult()
.topDocs(
new TopDocsAndMaxScore(new TopDocs(new TotalHits(0, TotalHits.Relation.EQUAL_TO), Lucene.EMPTY_SCORE_DOCS), Float.NaN),
new DocValueFormat[0]
);
return;
}
if (LOGGER.isTraceEnabled()) {
LOGGER.trace("{}", new SearchContextSourcePrinter(searchContext));
}
// Pre-process aggregations as late as possible. In the case of a DFS_Q_T_F
// request, preProcess is called on the DFS phase phase, this is why we pre-process them
// here to make sure it happens during the QUERY phase
aggregationPhase.preProcess(searchContext);
boolean rescore = executeInternal(searchContext, queryPhaseSearcher);
if (rescore) { // only if we do a regular search
rescorePhase.execute(searchContext);
}
suggestPhase.execute(searchContext);
aggregationPhase.execute(searchContext);
if (searchContext.getProfilers() != null) {
ProfileShardResult shardResults = SearchProfileShardResults.buildShardResults(
searchContext.getProfilers(),
searchContext.request()
);
searchContext.queryResult().profileResults(shardResults);
}
}
/**
* In a package-private method so that it can be tested without having to
* wire everything (mapperService, etc.)
* @return whether the rescoring phase should be executed
*/
static boolean executeInternal(SearchContext searchContext) throws QueryPhaseExecutionException {
return executeInternal(searchContext, QueryPhase.DEFAULT_QUERY_PHASE_SEARCHER);
}
/**
* In a package-private method so that it can be tested without having to
* wire everything (mapperService, etc.)
* @return whether the rescoring phase should be executed
*/
static boolean executeInternal(SearchContext searchContext, QueryPhaseSearcher queryPhaseSearcher) throws QueryPhaseExecutionException {
final ContextIndexSearcher searcher = searchContext.searcher();
final IndexReader reader = searcher.getIndexReader();
QuerySearchResult queryResult = searchContext.queryResult();
queryResult.searchTimedOut(false);
try {
queryResult.from(searchContext.from());
queryResult.size(searchContext.size());
Query query = searchContext.query();
assert query == searcher.rewrite(query); // already rewritten
final ScrollContext scrollContext = searchContext.scrollContext();
if (scrollContext != null) {
if (scrollContext.totalHits == null) {
// first round
assert scrollContext.lastEmittedDoc == null;
// there is not much that we can optimize here since we want to collect all
// documents in order to get the total number of hits
} else {
final ScoreDoc after = scrollContext.lastEmittedDoc;
if (returnsDocsInOrder(query, searchContext.sort())) {
// now this gets interesting: since we sort in index-order, we can directly
// skip to the desired doc
if (after != null) {
query = new BooleanQuery.Builder().add(query, BooleanClause.Occur.MUST)
.add(new MinDocQuery(after.doc + 1), BooleanClause.Occur.FILTER)
.build();
}
// ... and stop collecting after ${size} matches
searchContext.terminateAfter(searchContext.size());
} else if (canEarlyTerminate(reader, searchContext.sort())) {
// now this gets interesting: since the search sort is a prefix of the index sort, we can directly
// skip to the desired doc
if (after != null) {
query = new BooleanQuery.Builder().add(query, BooleanClause.Occur.MUST)
.add(new SearchAfterSortedDocQuery(searchContext.sort().sort, (FieldDoc) after), BooleanClause.Occur.FILTER)
.build();
}
}
}
}
final LinkedList<QueryCollectorContext> collectors = new LinkedList<>();
// whether the chain contains a collector that filters documents
boolean hasFilterCollector = false;
if (searchContext.terminateAfter() != SearchContext.DEFAULT_TERMINATE_AFTER) {
// add terminate_after before the filter collectors
// it will only be applied on documents accepted by these filter collectors
collectors.add(createEarlyTerminationCollectorContext(searchContext.terminateAfter()));
// this collector can filter documents during the collection
hasFilterCollector = true;
}
if (searchContext.parsedPostFilter() != null) {
// add post filters before aggregations
// it will only be applied to top hits
collectors.add(createFilteredCollectorContext(searcher, searchContext.parsedPostFilter().query()));
// this collector can filter documents during the collection
hasFilterCollector = true;
}
if (searchContext.queryCollectorManagers().isEmpty() == false) {
// plug in additional collectors, like aggregations
collectors.add(createMultiCollectorContext(searchContext.queryCollectorManagers().values()));
}
if (searchContext.minimumScore() != null) {
// apply the minimum score after multi collector so we filter aggs as well
collectors.add(createMinScoreCollectorContext(searchContext.minimumScore()));
// this collector can filter documents during the collection
hasFilterCollector = true;
}
boolean timeoutSet = scrollContext == null
&& searchContext.timeout() != null
&& searchContext.timeout().equals(SearchService.NO_TIMEOUT) == false;
final Runnable timeoutRunnable;
if (timeoutSet) {
timeoutRunnable = searcher.addQueryCancellation(
createQueryTimeoutChecker(searchContext::getRelativeTimeInMillis, searchContext.timeout().millis())
);
} else {
timeoutRunnable = null;
}
if (searchContext.lowLevelCancellation()) {
searcher.addQueryCancellation(() -> {
SearchShardTask task = searchContext.getTask();
if (task != null && task.isCancelled()) {
throw new TaskCancelledException("cancelled task with reason: " + task.getReasonCancelled());
}
});
}
try {
boolean shouldRescore = queryPhaseSearcher.searchWith(
searchContext,
searcher,
query,
collectors,
hasFilterCollector,
timeoutSet
);
ExecutorService executor = searchContext.indexShard().getThreadPool().executor(ThreadPool.Names.SEARCH);
if (executor instanceof EWMATrackingThreadPoolExecutor) {
final EWMATrackingThreadPoolExecutor rExecutor = (EWMATrackingThreadPoolExecutor) executor;
queryResult.nodeQueueSize(rExecutor.getCurrentQueueSize());
queryResult.serviceTimeEWMA((long) rExecutor.getTaskExecutionEWMA());
}
return shouldRescore;
} finally {
// Search phase has finished, no longer need to check for timeout
// otherwise aggregation phase might get cancelled.
if (timeoutRunnable != null) {
searcher.removeQueryCancellation(timeoutRunnable);
}
}
} catch (Exception e) {
throw new QueryPhaseExecutionException(searchContext.shardTarget(), "Failed to execute main query", e);
}
}
/**
* Create runnable which throws {@link TimeExceededException} when the runnable is called after timeout + runnable creation time
* exceeds currentTime
* @param relativeMsTimeSupplier supplier of currentTime in milliseconds which is okay to be cached
* @param timeout the max time duration in milliseconds before throwing {@link TimeExceededException}
* @return the created runnable
*/
static Runnable createQueryTimeoutChecker(LongSupplier relativeMsTimeSupplier, final long timeout) {
/* for startTime, relative non-cached time must be used to prevent false positive timeouts.
* Using cached time for startTime will fail and produce false positive timeouts when maxTime = (startTime + timeout) falls in
* next time cache slot(s) AND time caching lifespan > passed timeout */
final long startTime = TimeValue.nsecToMSec(System.nanoTime());
final long maxTime = startTime + timeout;
return () -> {
/* As long as startTime is non cached time, using cached time here might only produce false negative timeouts within the time
* cache life span which is acceptable */
final long time = relativeMsTimeSupplier.getAsLong();
if (time > maxTime) {
throw new TimeExceededException();
}
};
}
private static boolean searchWithCollector(
SearchContext searchContext,
ContextIndexSearcher searcher,
Query query,
LinkedList<QueryCollectorContext> collectors,
boolean hasFilterCollector,
boolean timeoutSet
) throws IOException {
// create the top docs collector last when the other collectors are known
final TopDocsCollectorContext topDocsFactory = createTopDocsCollectorContext(searchContext, hasFilterCollector);
// add the top docs collector, the first collector context in the chain
collectors.addFirst(topDocsFactory);
final Collector queryCollector;
if (searchContext.getProfilers() != null) {
InternalProfileCollector profileCollector = QueryCollectorContext.createQueryCollectorWithProfiler(collectors);
searchContext.getProfilers().getCurrentQueryProfiler().setCollector(profileCollector);
queryCollector = profileCollector;
} else {
queryCollector = QueryCollectorContext.createQueryCollector(collectors);
}
QuerySearchResult queryResult = searchContext.queryResult();
try {
searcher.search(query, queryCollector);
} catch (EarlyTerminatingCollector.EarlyTerminationException e) {
queryResult.terminatedEarly(true);
} catch (TimeExceededException e) {
assert timeoutSet : "TimeExceededException thrown even though timeout wasn't set";
if (searchContext.request().allowPartialSearchResults() == false) {
// Can't rethrow TimeExceededException because not serializable
throw new QueryPhaseExecutionException(searchContext.shardTarget(), "Time exceeded");
}
queryResult.searchTimedOut(true);
}
if (searchContext.terminateAfter() != SearchContext.DEFAULT_TERMINATE_AFTER && queryResult.terminatedEarly() == null) {
queryResult.terminatedEarly(false);
}
for (QueryCollectorContext ctx : collectors) {
ctx.postProcess(queryResult);
}
return topDocsFactory.shouldRescore();
}
/**
* Returns true if the provided <code>query</code> returns docs in index order (internal doc ids).
* @param query The query to execute
* @param sf The query sort
*/
private static boolean returnsDocsInOrder(Query query, SortAndFormats sf) {
if (sf == null || Sort.RELEVANCE.equals(sf.sort)) {
// sort by score
// queries that return constant scores will return docs in index
// order since Lucene tie-breaks on the doc id
return query.getClass() == ConstantScoreQuery.class || query.getClass() == MatchAllDocsQuery.class;
} else {
return Sort.INDEXORDER.equals(sf.sort);
}
}
/**
* Returns whether collection within the provided <code>reader</code> can be early-terminated if it sorts
* with <code>sortAndFormats</code>.
**/
private static boolean canEarlyTerminate(IndexReader reader, SortAndFormats sortAndFormats) {
if (sortAndFormats == null || sortAndFormats.sort == null) {
return false;
}
final Sort sort = sortAndFormats.sort;
for (LeafReaderContext ctx : reader.leaves()) {
Sort indexSort = ctx.reader().getMetaData().getSort();
if (indexSort == null || Lucene.canEarlyTerminate(sort, indexSort) == false) {
return false;
}
}
return true;
}
/**
* The exception being raised when search timeout is reached.
*
* @opensearch.internal
*/
public static class TimeExceededException extends RuntimeException {
private static final long serialVersionUID = 1L;
}
/**
* Default {@link QueryPhaseSearcher} implementation which delegates to the {@link QueryPhase}.
*
* @opensearch.internal
*/
public static class DefaultQueryPhaseSearcher implements QueryPhaseSearcher {
/**
* Please use {@link QueryPhase#DEFAULT_QUERY_PHASE_SEARCHER}
*/
protected DefaultQueryPhaseSearcher() {}
@Override
public boolean searchWith(
SearchContext searchContext,
ContextIndexSearcher searcher,
Query query,
LinkedList<QueryCollectorContext> collectors,
boolean hasFilterCollector,
boolean hasTimeout
) throws IOException {
return searchWithCollector(searchContext, searcher, query, collectors, hasFilterCollector, hasTimeout);
}
protected boolean searchWithCollector(
SearchContext searchContext,
ContextIndexSearcher searcher,
Query query,
LinkedList<QueryCollectorContext> collectors,
boolean hasFilterCollector,
boolean hasTimeout
) throws IOException {
return QueryPhase.searchWithCollector(searchContext, searcher, query, collectors, hasFilterCollector, hasTimeout);
}
}
}