-
Notifications
You must be signed in to change notification settings - Fork 24.3k
/
SearchService.java
1424 lines (1302 loc) · 70 KB
/
SearchService.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
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License
* 2.0 and the Server Side Public License, v 1; you may not use this file except
* in compliance with, at your election, the Elastic License 2.0 or the Server
* Side Public License, v 1.
*/
package org.elasticsearch.search;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.apache.lucene.search.FieldDoc;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.TopDocs;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionRunnable;
import org.elasticsearch.action.OriginalIndices;
import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchShardTask;
import org.elasticsearch.action.search.SearchType;
import org.elasticsearch.action.support.TransportActions;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.CheckedSupplier;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.breaker.CircuitBreaker;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.CollectionUtils;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.common.util.concurrent.ConcurrentMapLong;
import org.elasticsearch.core.internal.io.IOUtils;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.query.CoordinatorRewriteContextProvider;
import org.elasticsearch.index.query.InnerHitContextBuilder;
import org.elasticsearch.index.query.MatchAllQueryBuilder;
import org.elasticsearch.index.query.MatchNoneQueryBuilder;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryRewriteContext;
import org.elasticsearch.index.query.SearchExecutionContext;
import org.elasticsearch.index.query.Rewriteable;
import org.elasticsearch.index.shard.IndexEventListener;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.SearchOperationListener;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason;
import org.elasticsearch.node.ResponseCollectorService;
import org.elasticsearch.script.FieldScript;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.aggregations.AggregationInitializationException;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregation.ReduceContext;
import org.elasticsearch.search.aggregations.MultiBucketConsumerService;
import org.elasticsearch.search.aggregations.SearchContextAggregations;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator.PipelineTree;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.AggregationContext.ProductionAggregationContext;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.collapse.CollapseContext;
import org.elasticsearch.search.dfs.DfsPhase;
import org.elasticsearch.search.dfs.DfsSearchResult;
import org.elasticsearch.search.fetch.FetchPhase;
import org.elasticsearch.search.fetch.FetchSearchResult;
import org.elasticsearch.search.fetch.QueryFetchSearchResult;
import org.elasticsearch.search.fetch.ScrollQueryFetchSearchResult;
import org.elasticsearch.search.fetch.ShardFetchRequest;
import org.elasticsearch.search.fetch.subphase.FetchDocValuesContext;
import org.elasticsearch.search.fetch.subphase.FetchFieldsContext;
import org.elasticsearch.search.fetch.subphase.ScriptFieldsContext.ScriptField;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder;
import org.elasticsearch.search.internal.AliasFilter;
import org.elasticsearch.search.internal.InternalScrollSearchRequest;
import org.elasticsearch.search.internal.LegacyReaderContext;
import org.elasticsearch.search.internal.ReaderContext;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.internal.ShardSearchContextId;
import org.elasticsearch.search.internal.ShardSearchRequest;
import org.elasticsearch.search.internal.SubSearchContext;
import org.elasticsearch.search.lookup.SearchLookup;
import org.elasticsearch.search.profile.Profilers;
import org.elasticsearch.search.query.QueryPhase;
import org.elasticsearch.search.query.QuerySearchRequest;
import org.elasticsearch.search.query.QuerySearchResult;
import org.elasticsearch.search.query.ScrollQuerySearchResult;
import org.elasticsearch.search.rescore.RescorerBuilder;
import org.elasticsearch.search.searchafter.SearchAfterBuilder;
import org.elasticsearch.search.sort.FieldSortBuilder;
import org.elasticsearch.search.sort.MinAndMax;
import org.elasticsearch.search.sort.SortAndFormats;
import org.elasticsearch.search.sort.SortBuilder;
import org.elasticsearch.search.suggest.Suggest;
import org.elasticsearch.search.suggest.completion.CompletionSuggestion;
import org.elasticsearch.threadpool.Scheduler.Cancellable;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.threadpool.ThreadPool.Names;
import org.elasticsearch.transport.TransportRequest;
import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executor;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.LongSupplier;
import static org.elasticsearch.common.unit.TimeValue.timeValueHours;
import static org.elasticsearch.common.unit.TimeValue.timeValueMillis;
import static org.elasticsearch.common.unit.TimeValue.timeValueMinutes;
public class SearchService extends AbstractLifecycleComponent implements IndexEventListener {
private static final Logger logger = LogManager.getLogger(SearchService.class);
// we can have 5 minutes here, since we make sure to clean with search requests and when shard/index closes
public static final Setting<TimeValue> DEFAULT_KEEPALIVE_SETTING =
Setting.positiveTimeSetting("search.default_keep_alive", timeValueMinutes(5), Property.NodeScope, Property.Dynamic);
public static final Setting<TimeValue> MAX_KEEPALIVE_SETTING =
Setting.positiveTimeSetting("search.max_keep_alive", timeValueHours(24), Property.NodeScope, Property.Dynamic);
public static final Setting<TimeValue> KEEPALIVE_INTERVAL_SETTING =
Setting.positiveTimeSetting("search.keep_alive_interval", timeValueMinutes(1), Property.NodeScope);
public static final Setting<Boolean> ALLOW_EXPENSIVE_QUERIES =
Setting.boolSetting("search.allow_expensive_queries", true, Property.NodeScope, Property.Dynamic);
/**
* Enables low-level, frequent search cancellation checks. Enabling low-level checks will make long running searches to react
* to the cancellation request faster. It will produce more cancellation checks but benchmarking has shown these did not
* noticeably slow down searches.
*/
public static final Setting<Boolean> LOW_LEVEL_CANCELLATION_SETTING =
Setting.boolSetting("search.low_level_cancellation", true, Property.Dynamic, Property.NodeScope);
public static final TimeValue NO_TIMEOUT = timeValueMillis(-1);
public static final Setting<TimeValue> DEFAULT_SEARCH_TIMEOUT_SETTING =
Setting.timeSetting("search.default_search_timeout", NO_TIMEOUT, Property.Dynamic, Property.NodeScope);
public static final Setting<Boolean> DEFAULT_ALLOW_PARTIAL_SEARCH_RESULTS =
Setting.boolSetting("search.default_allow_partial_results", true, Property.Dynamic, Property.NodeScope);
public static final Setting<Integer> MAX_OPEN_SCROLL_CONTEXT =
Setting.intSetting("search.max_open_scroll_context", 500, 0, Property.Dynamic, Property.NodeScope);
public static final int DEFAULT_SIZE = 10;
public static final int DEFAULT_FROM = 0;
private final ThreadPool threadPool;
private final ClusterService clusterService;
private final IndicesService indicesService;
private final ScriptService scriptService;
private final ResponseCollectorService responseCollectorService;
private final BigArrays bigArrays;
private final DfsPhase dfsPhase = new DfsPhase();
private final QueryPhase queryPhase;
private final FetchPhase fetchPhase;
private volatile long defaultKeepAlive;
private volatile long maxKeepAlive;
private volatile TimeValue defaultSearchTimeout;
private volatile boolean defaultAllowPartialSearchResults;
private volatile boolean lowLevelCancellation;
private volatile int maxOpenScrollContext;
private final Cancellable keepAliveReaper;
private final AtomicLong idGenerator = new AtomicLong();
private final ConcurrentMapLong<ReaderContext> activeReaders = ConcurrentCollections.newConcurrentMapLongWithAggressiveConcurrency();
private final MultiBucketConsumerService multiBucketConsumerService;
private final AtomicInteger openScrollContexts = new AtomicInteger();
private final String sessionId = UUIDs.randomBase64UUID();
public SearchService(ClusterService clusterService, IndicesService indicesService,
ThreadPool threadPool, ScriptService scriptService, BigArrays bigArrays, FetchPhase fetchPhase,
ResponseCollectorService responseCollectorService, CircuitBreakerService circuitBreakerService) {
Settings settings = clusterService.getSettings();
this.threadPool = threadPool;
this.clusterService = clusterService;
this.indicesService = indicesService;
this.scriptService = scriptService;
this.responseCollectorService = responseCollectorService;
this.bigArrays = bigArrays;
this.queryPhase = new QueryPhase();
this.fetchPhase = fetchPhase;
this.multiBucketConsumerService = new MultiBucketConsumerService(clusterService, settings,
circuitBreakerService.getBreaker(CircuitBreaker.REQUEST));
TimeValue keepAliveInterval = KEEPALIVE_INTERVAL_SETTING.get(settings);
setKeepAlives(DEFAULT_KEEPALIVE_SETTING.get(settings), MAX_KEEPALIVE_SETTING.get(settings));
clusterService.getClusterSettings().addSettingsUpdateConsumer(DEFAULT_KEEPALIVE_SETTING, MAX_KEEPALIVE_SETTING,
this::setKeepAlives, this::validateKeepAlives);
this.keepAliveReaper = threadPool.scheduleWithFixedDelay(new Reaper(), keepAliveInterval, Names.SAME);
defaultSearchTimeout = DEFAULT_SEARCH_TIMEOUT_SETTING.get(settings);
clusterService.getClusterSettings().addSettingsUpdateConsumer(DEFAULT_SEARCH_TIMEOUT_SETTING, this::setDefaultSearchTimeout);
defaultAllowPartialSearchResults = DEFAULT_ALLOW_PARTIAL_SEARCH_RESULTS.get(settings);
clusterService.getClusterSettings().addSettingsUpdateConsumer(DEFAULT_ALLOW_PARTIAL_SEARCH_RESULTS,
this::setDefaultAllowPartialSearchResults);
maxOpenScrollContext = MAX_OPEN_SCROLL_CONTEXT.get(settings);
clusterService.getClusterSettings().addSettingsUpdateConsumer(MAX_OPEN_SCROLL_CONTEXT, this::setMaxOpenScrollContext);
lowLevelCancellation = LOW_LEVEL_CANCELLATION_SETTING.get(settings);
clusterService.getClusterSettings().addSettingsUpdateConsumer(LOW_LEVEL_CANCELLATION_SETTING, this::setLowLevelCancellation);
}
private void validateKeepAlives(TimeValue defaultKeepAlive, TimeValue maxKeepAlive) {
if (defaultKeepAlive.millis() > maxKeepAlive.millis()) {
throw new IllegalArgumentException("Default keep alive setting for request [" + DEFAULT_KEEPALIVE_SETTING.getKey() + "]" +
" should be smaller than max keep alive [" + MAX_KEEPALIVE_SETTING.getKey() + "], " +
"was (" + defaultKeepAlive + " > " + maxKeepAlive + ")");
}
}
private void setKeepAlives(TimeValue defaultKeepAlive, TimeValue maxKeepAlive) {
validateKeepAlives(defaultKeepAlive, maxKeepAlive);
this.defaultKeepAlive = defaultKeepAlive.millis();
this.maxKeepAlive = maxKeepAlive.millis();
}
private void setDefaultSearchTimeout(TimeValue defaultSearchTimeout) {
this.defaultSearchTimeout = defaultSearchTimeout;
}
private void setDefaultAllowPartialSearchResults(boolean defaultAllowPartialSearchResults) {
this.defaultAllowPartialSearchResults = defaultAllowPartialSearchResults;
}
public boolean defaultAllowPartialSearchResults() {
return defaultAllowPartialSearchResults;
}
private void setMaxOpenScrollContext(int maxOpenScrollContext) {
this.maxOpenScrollContext = maxOpenScrollContext;
}
private void setLowLevelCancellation(Boolean lowLevelCancellation) {
this.lowLevelCancellation = lowLevelCancellation;
}
@Override
public void afterIndexRemoved(Index index, IndexSettings indexSettings, IndexRemovalReason reason) {
// once an index is removed due to deletion or closing, we can just clean up all the pending search context information
// if we then close all the contexts we can get some search failures along the way which are not expected.
// it's fine to keep the contexts open if the index is still "alive"
// unfortunately we don't have a clear way to signal today why an index is closed.
// to release memory and let references to the filesystem go etc.
if (reason == IndexRemovalReason.DELETED || reason == IndexRemovalReason.CLOSED || reason == IndexRemovalReason.REOPENED) {
freeAllContextForIndex(index);
}
}
protected void putReaderContext(ReaderContext context) {
final ReaderContext previous = activeReaders.put(context.id().getId(), context);
assert previous == null;
// ensure that if we race against afterIndexRemoved, we remove the context from the active list.
// this is important to ensure store can be cleaned up, in particular if the search is a scroll with a long timeout.
final Index index = context.indexShard().shardId().getIndex();
if (indicesService.hasIndex(index) == false) {
removeReaderContext(context.id().getId());
throw new IndexNotFoundException(index);
}
}
protected ReaderContext removeReaderContext(long id) {
return activeReaders.remove(id);
}
@Override
protected void doStart() {
}
@Override
protected void doStop() {
for (final ReaderContext context : activeReaders.values()) {
freeReaderContext(context.id());
}
}
@Override
protected void doClose() {
doStop();
keepAliveReaper.cancel();
}
public void executeDfsPhase(ShardSearchRequest request, SearchShardTask task, ActionListener<SearchPhaseResult> listener) {
final IndexShard shard = getShard(request);
rewriteAndFetchShardRequest(shard, request, new ActionListener<ShardSearchRequest>() {
@Override
public void onResponse(ShardSearchRequest rewritten) {
// fork the execution in the search thread pool
runAsync(getExecutor(shard), () -> executeDfsPhase(request, task), listener);
}
@Override
public void onFailure(Exception exc) {
listener.onFailure(exc);
}
});
}
private DfsSearchResult executeDfsPhase(ShardSearchRequest request, SearchShardTask task) throws IOException {
ReaderContext readerContext = createOrGetReaderContext(request);
try (Releasable ignored = readerContext.markAsUsed(getKeepAlive(request));
SearchContext context = createContext(readerContext, request, task, true)) {
dfsPhase.execute(context);
return context.dfsResult();
} catch (Exception e) {
logger.trace("Dfs phase failed", e);
processFailure(readerContext, e);
throw e;
}
}
/**
* Try to load the query results from the cache or execute the query phase directly if the cache cannot be used.
*/
private void loadOrExecuteQueryPhase(final ShardSearchRequest request, final SearchContext context) throws Exception {
final boolean canCache = indicesService.canCache(request, context);
context.getSearchExecutionContext().freezeContext();
if (canCache) {
indicesService.loadIntoContext(request, context, queryPhase);
} else {
queryPhase.execute(context);
}
}
public void executeQueryPhase(ShardSearchRequest request, SearchShardTask task, ActionListener<SearchPhaseResult> listener) {
assert request.canReturnNullResponseIfMatchNoDocs() == false || request.numberOfShards() > 1
: "empty responses require more than one shard";
final IndexShard shard = getShard(request);
rewriteAndFetchShardRequest(shard, request, new ActionListener<ShardSearchRequest>() {
@Override
public void onResponse(ShardSearchRequest orig) {
// check if we can shortcut the query phase entirely.
if (orig.canReturnNullResponseIfMatchNoDocs()) {
assert orig.scroll() == null;
final CanMatchResponse canMatchResp;
try {
ShardSearchRequest clone = new ShardSearchRequest(orig);
canMatchResp = canMatch(clone, false);
} catch (Exception exc) {
listener.onFailure(exc);
return;
}
if (canMatchResp.canMatch == false) {
listener.onResponse(QuerySearchResult.nullInstance());
return;
}
}
// fork the execution in the search thread pool
runAsync(getExecutor(shard), () -> executeQueryPhase(orig, task), listener);
}
@Override
public void onFailure(Exception exc) {
listener.onFailure(exc);
}
});
}
private IndexShard getShard(ShardSearchRequest request) {
final ShardSearchContextId contextId = request.readerId();
if (contextId != null) {
assert contextId.getSessionId().isEmpty() == false : request;
if (sessionId.equals(contextId.getSessionId())) {
final ReaderContext readerContext = activeReaders.get(contextId.getId());
if (readerContext != null) {
return readerContext.indexShard();
}
}
}
return indicesService.indexServiceSafe(request.shardId().getIndex()).getShard(request.shardId().id());
}
private <T> void runAsync(Executor executor, CheckedSupplier<T, Exception> executable, ActionListener<T> listener) {
executor.execute(ActionRunnable.supply(listener, executable::get));
}
private SearchPhaseResult executeQueryPhase(ShardSearchRequest request, SearchShardTask task) throws Exception {
final ReaderContext readerContext = createOrGetReaderContext(request);
try (Releasable ignored = readerContext.markAsUsed(getKeepAlive(request));
SearchContext context = createContext(readerContext, request, task, true)) {
final long afterQueryTime;
try (SearchOperationListenerExecutor executor = new SearchOperationListenerExecutor(context)) {
loadOrExecuteQueryPhase(request, context);
if (context.queryResult().hasSearchContext() == false && readerContext.singleSession()) {
freeReaderContext(readerContext.id());
}
afterQueryTime = executor.success();
}
if (request.numberOfShards() == 1) {
return executeFetchPhase(readerContext, context, afterQueryTime);
} else {
// Pass the rescoreDocIds to the queryResult to send them the coordinating node and receive them back in the fetch phase.
// We also pass the rescoreDocIds to the LegacyReaderContext in case the search state needs to stay in the data node.
final RescoreDocIds rescoreDocIds = context.rescoreDocIds();
context.queryResult().setRescoreDocIds(rescoreDocIds);
readerContext.setRescoreDocIds(rescoreDocIds);
return context.queryResult();
}
} catch (Exception e) {
// execution exception can happen while loading the cache, strip it
if (e instanceof ExecutionException) {
e = (e.getCause() == null || e.getCause() instanceof Exception) ?
(Exception) e.getCause() : new ElasticsearchException(e.getCause());
}
logger.trace("Query phase failed", e);
processFailure(readerContext, e);
throw e;
}
}
private QueryFetchSearchResult executeFetchPhase(ReaderContext reader, SearchContext context, long afterQueryTime) {
try (SearchOperationListenerExecutor executor = new SearchOperationListenerExecutor(context, true, afterQueryTime)){
shortcutDocIdsToLoad(context);
fetchPhase.execute(context);
if (reader.singleSession()) {
freeReaderContext(reader.id());
}
executor.success();
}
return new QueryFetchSearchResult(context.queryResult(), context.fetchResult());
}
public void executeQueryPhase(InternalScrollSearchRequest request,
SearchShardTask task,
ActionListener<ScrollQuerySearchResult> listener) {
final LegacyReaderContext readerContext = (LegacyReaderContext) findReaderContext(request.contextId(), request);
final Releasable markAsUsed;
try {
markAsUsed = readerContext.markAsUsed(getScrollKeepAlive(request.scroll()));
} catch (Exception e) {
// We need to release the reader context of the scroll when we hit any exception (here the keep_alive can be too large)
freeReaderContext(readerContext.id());
throw e;
}
runAsync(getExecutor(readerContext.indexShard()), () -> {
final ShardSearchRequest shardSearchRequest = readerContext.getShardSearchRequest(null);
try (SearchContext searchContext = createContext(readerContext, shardSearchRequest, task, false);
SearchOperationListenerExecutor executor = new SearchOperationListenerExecutor(searchContext)) {
searchContext.searcher().setAggregatedDfs(readerContext.getAggregatedDfs(null));
processScroll(request, readerContext, searchContext);
queryPhase.execute(searchContext);
executor.success();
readerContext.setRescoreDocIds(searchContext.rescoreDocIds());
return new ScrollQuerySearchResult(searchContext.queryResult(), searchContext.shardTarget());
} catch (Exception e) {
logger.trace("Query phase failed", e);
// we handle the failure in the failure listener below
throw e;
}
}, wrapFailureListener(listener, readerContext, markAsUsed));
}
public void executeQueryPhase(QuerySearchRequest request, SearchShardTask task, ActionListener<QuerySearchResult> listener) {
final ReaderContext readerContext = findReaderContext(request.contextId(), request.shardSearchRequest());
final ShardSearchRequest shardSearchRequest = readerContext.getShardSearchRequest(request.shardSearchRequest());
final Releasable markAsUsed = readerContext.markAsUsed(getKeepAlive(shardSearchRequest));
runAsync(getExecutor(readerContext.indexShard()), () -> {
readerContext.setAggregatedDfs(request.dfs());
try (SearchContext searchContext = createContext(readerContext, shardSearchRequest, task, true);
SearchOperationListenerExecutor executor = new SearchOperationListenerExecutor(searchContext)) {
searchContext.searcher().setAggregatedDfs(request.dfs());
queryPhase.execute(searchContext);
if (searchContext.queryResult().hasSearchContext() == false && readerContext.singleSession()) {
// no hits, we can release the context since there will be no fetch phase
freeReaderContext(readerContext.id());
}
executor.success();
// Pass the rescoreDocIds to the queryResult to send them the coordinating node and receive them back in the fetch phase.
// We also pass the rescoreDocIds to the LegacyReaderContext in case the search state needs to stay in the data node.
final RescoreDocIds rescoreDocIds = searchContext.rescoreDocIds();
searchContext.queryResult().setRescoreDocIds(rescoreDocIds);
readerContext.setRescoreDocIds(rescoreDocIds);
return searchContext.queryResult();
} catch (Exception e) {
assert TransportActions.isShardNotAvailableException(e) == false : new AssertionError(e);
logger.trace("Query phase failed", e);
// we handle the failure in the failure listener below
throw e;
}
}, wrapFailureListener(listener, readerContext, markAsUsed));
}
private Executor getExecutor(IndexShard indexShard) {
assert indexShard != null;
final String executorName;
if (indexShard.isSystem()) {
executorName = Names.SYSTEM_READ;
} else if (indexShard.indexSettings().isSearchThrottled()) {
executorName = Names.SEARCH_THROTTLED;
} else {
executorName = Names.SEARCH;
}
return threadPool.executor(executorName);
}
public void executeFetchPhase(InternalScrollSearchRequest request, SearchShardTask task,
ActionListener<ScrollQueryFetchSearchResult> listener) {
final LegacyReaderContext readerContext = (LegacyReaderContext) findReaderContext(request.contextId(), request);
final Releasable markAsUsed;
try {
markAsUsed = readerContext.markAsUsed(getScrollKeepAlive(request.scroll()));
} catch (Exception e) {
// We need to release the reader context of the scroll when we hit any exception (here the keep_alive can be too large)
freeReaderContext(readerContext.id());
throw e;
}
runAsync(getExecutor(readerContext.indexShard()), () -> {
final ShardSearchRequest shardSearchRequest = readerContext.getShardSearchRequest(null);
try (SearchContext searchContext = createContext(readerContext, shardSearchRequest, task, false);
SearchOperationListenerExecutor executor = new SearchOperationListenerExecutor(searchContext)) {
searchContext.assignRescoreDocIds(readerContext.getRescoreDocIds(null));
searchContext.searcher().setAggregatedDfs(readerContext.getAggregatedDfs(null));
processScroll(request, readerContext, searchContext);
queryPhase.execute(searchContext);
final long afterQueryTime = executor.success();
QueryFetchSearchResult fetchSearchResult = executeFetchPhase(readerContext, searchContext, afterQueryTime);
return new ScrollQueryFetchSearchResult(fetchSearchResult, searchContext.shardTarget());
} catch (Exception e) {
assert TransportActions.isShardNotAvailableException(e) == false : new AssertionError(e);
logger.trace("Fetch phase failed", e);
// we handle the failure in the failure listener below
throw e;
}
}, wrapFailureListener(listener, readerContext, markAsUsed));
}
public void executeFetchPhase(ShardFetchRequest request, SearchShardTask task, ActionListener<FetchSearchResult> listener) {
final ReaderContext readerContext = findReaderContext(request.contextId(), request);
final ShardSearchRequest shardSearchRequest = readerContext.getShardSearchRequest(request.getShardSearchRequest());
final Releasable markAsUsed = readerContext.markAsUsed(getKeepAlive(shardSearchRequest));
runAsync(getExecutor(readerContext.indexShard()), () -> {
try (SearchContext searchContext = createContext(readerContext, shardSearchRequest, task, false)) {
if (request.lastEmittedDoc() != null) {
searchContext.scrollContext().lastEmittedDoc = request.lastEmittedDoc();
}
searchContext.assignRescoreDocIds(readerContext.getRescoreDocIds(request.getRescoreDocIds()));
searchContext.searcher().setAggregatedDfs(readerContext.getAggregatedDfs(request.getAggregatedDfs()));
searchContext.docIdsToLoad(request.docIds(), request.docIdsSize());
try (SearchOperationListenerExecutor executor =
new SearchOperationListenerExecutor(searchContext, true, System.nanoTime())) {
fetchPhase.execute(searchContext);
if (readerContext.singleSession()) {
freeReaderContext(request.contextId());
}
executor.success();
}
return searchContext.fetchResult();
} catch (Exception e) {
assert TransportActions.isShardNotAvailableException(e) == false : new AssertionError(e);
// we handle the failure in the failure listener below
throw e;
}
}, wrapFailureListener(listener, readerContext, markAsUsed));
}
private ReaderContext findReaderContext(ShardSearchContextId id, TransportRequest request) throws SearchContextMissingException {
if (sessionId.equals(id.getSessionId()) == false && id.getSessionId().isEmpty() == false) {
throw new SearchContextMissingException(id);
}
final ReaderContext reader = activeReaders.get(id.getId());
if (reader == null) {
throw new SearchContextMissingException(id);
}
try {
reader.validate(request);
} catch (Exception exc) {
processFailure(reader, exc);
throw exc;
}
return reader;
}
final ReaderContext createOrGetReaderContext(ShardSearchRequest request) {
if (request.readerId() != null) {
assert request.scroll() == null : "scroll can't be used with pit";
try {
return findReaderContext(request.readerId(), request);
} catch (SearchContextMissingException e) {
final String searcherId = request.readerId().getSearcherId();
if (searcherId == null) {
throw e;
}
final IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex());
final IndexShard shard = indexService.getShard(request.shardId().id());
final Engine.SearcherSupplier searcherSupplier = shard.acquireSearcherSupplier();
if (searcherId.equals(searcherSupplier.getSearcherId()) == false) {
searcherSupplier.close();
throw e;
}
return createAndPutReaderContext(request, indexService, shard, searcherSupplier, defaultKeepAlive);
}
} else {
final long keepAliveInMillis = getKeepAlive(request);
final IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex());
final IndexShard shard = indexService.getShard(request.shardId().id());
final Engine.SearcherSupplier searcherSupplier = shard.acquireSearcherSupplier();
return createAndPutReaderContext(request, indexService, shard, searcherSupplier, keepAliveInMillis);
}
}
final ReaderContext createAndPutReaderContext(ShardSearchRequest request, IndexService indexService, IndexShard shard,
Engine.SearcherSupplier reader, long keepAliveInMillis) {
ReaderContext readerContext = null;
Releasable decreaseScrollContexts = null;
try {
if (request.scroll() != null) {
decreaseScrollContexts = openScrollContexts::decrementAndGet;
if (openScrollContexts.incrementAndGet() > 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 ShardSearchContextId id = new ShardSearchContextId(sessionId, idGenerator.incrementAndGet());
// Previously, the search states are stored in ReaderContext on data nodes. Since 7.10, they are now
// sent to the coordinating node in QuerySearchResult and the coordinating node then sends them back
// in ShardFetchSearchRequest. We must keep the search states in ReaderContext unless the coordinating
// node is guaranteed to send them back in the fetch phase.
// Three cases that we have to keep the search states in ReaderContext:
// 1. Scroll requests
// 2. The coordinating node or a proxy node (i.e. CCS) is on the old version. The `channelVersion`
// of ShardSearchRequest, which is the minimum version of nodes that the request has been passed,
// can be used to determine this.
// 3. Any node on the cluster is on the old version. This extra check is to avoid a situation where a
// ShardSearchRequest is sent via a new proxy node, but a ShardFetchSearchRequest on an old proxy node.
//
// Note that it's ok to keep the search states in ReaderContext even when the coordinating node also sends
// them back in the fetch phase and it only happens in a mixed cluster.
if (request.scroll() != null ||
request.getChannelVersion().before(Version.V_7_12_1) ||
clusterService.state().nodes().getMinNodeVersion().before(Version.V_7_12_1)) {
readerContext = new LegacyReaderContext(id, indexService, shard, reader, request, keepAliveInMillis);
if (request.scroll() != null) {
readerContext.addOnClose(decreaseScrollContexts);
decreaseScrollContexts = null;
}
} else {
readerContext = new ReaderContext(id, indexService, shard, reader, keepAliveInMillis, true);
}
reader = null;
final ReaderContext finalReaderContext = readerContext;
final SearchOperationListener searchOperationListener = shard.getSearchOperationListener();
searchOperationListener.onNewReaderContext(finalReaderContext);
if (finalReaderContext.scrollContext() != null) {
searchOperationListener.onNewScrollContext(finalReaderContext);
}
readerContext.addOnClose(() -> {
try {
if (finalReaderContext.scrollContext() != null) {
searchOperationListener.onFreeScrollContext(finalReaderContext);
}
} finally {
searchOperationListener.onFreeReaderContext(finalReaderContext);
}
});
putReaderContext(finalReaderContext);
readerContext = null;
return finalReaderContext;
} finally {
Releasables.close(reader, readerContext, decreaseScrollContexts);
}
}
/**
* Opens the reader context for given shardId. The newly opened reader context will be keep
* until the {@code keepAlive} elapsed unless it is manually released.
*/
public void openReaderContext(ShardId shardId, TimeValue keepAlive, ActionListener<ShardSearchContextId> listener) {
checkKeepAliveLimit(keepAlive.millis());
final IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
final IndexShard shard = indexService.getShard(shardId.id());
final SearchOperationListener searchOperationListener = shard.getSearchOperationListener();
shard.awaitShardSearchActive(ignored -> {
Engine.SearcherSupplier searcherSupplier = null;
ReaderContext readerContext = null;
try {
searcherSupplier = shard.acquireSearcherSupplier();
final ShardSearchContextId id =
new ShardSearchContextId(sessionId, idGenerator.incrementAndGet(), searcherSupplier.getSearcherId());
readerContext = new ReaderContext(id, indexService, shard, searcherSupplier, keepAlive.millis(), false);
final ReaderContext finalReaderContext = readerContext;
searcherSupplier = null; // transfer ownership to reader context
searchOperationListener.onNewReaderContext(readerContext);
readerContext.addOnClose(() -> searchOperationListener.onFreeReaderContext(finalReaderContext));
putReaderContext(readerContext);
readerContext = null;
listener.onResponse(finalReaderContext.id());
} catch (Exception exc) {
Releasables.closeWhileHandlingException(searcherSupplier, readerContext);
listener.onFailure(exc);
}
});
}
final SearchContext createContext(ReaderContext readerContext,
ShardSearchRequest request,
SearchShardTask task,
boolean includeAggregations) throws IOException {
final DefaultSearchContext context = createSearchContext(readerContext, request, defaultSearchTimeout);
try {
if (request.scroll() != null) {
context.scrollContext().scroll = request.scroll();
}
parseSource(context, request.source(), includeAggregations);
// if the from and size are still not set, default them
if (context.from() == -1) {
context.from(DEFAULT_FROM);
}
if (context.size() == -1) {
context.size(DEFAULT_SIZE);
}
context.setTask(task);
// pre process
queryPhase.preProcess(context);
} catch (Exception e) {
context.close();
throw e;
}
return context;
}
public DefaultSearchContext createSearchContext(ShardSearchRequest request, TimeValue timeout) throws IOException {
final IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex());
final IndexShard indexShard = indexService.getShard(request.shardId().getId());
final Engine.SearcherSupplier reader = indexShard.acquireSearcherSupplier();
final ShardSearchContextId id = new ShardSearchContextId(sessionId, idGenerator.incrementAndGet());
try (ReaderContext readerContext = new ReaderContext(id, indexService, indexShard, reader, -1L, true)) {
DefaultSearchContext searchContext = createSearchContext(readerContext, request, timeout);
searchContext.addReleasable(readerContext.markAsUsed(0L));
return searchContext;
}
}
private DefaultSearchContext createSearchContext(ReaderContext reader, ShardSearchRequest request, TimeValue timeout)
throws IOException {
boolean success = false;
DefaultSearchContext searchContext = null;
try {
SearchShardTarget shardTarget = new SearchShardTarget(clusterService.localNode().getId(),
reader.indexShard().shardId(), request.getClusterAlias(), OriginalIndices.NONE);
searchContext = new DefaultSearchContext(reader, request, shardTarget,
threadPool::relativeTimeInMillis, timeout, fetchPhase, lowLevelCancellation,
clusterService.state().nodes().getMinNodeVersion());
// we clone the query shard context here just for rewriting otherwise we
// might end up with incorrect state since we are using now() or script services
// during rewrite and normalized / evaluate templates etc.
SearchExecutionContext context = new SearchExecutionContext(searchContext.getSearchExecutionContext());
Rewriteable.rewrite(request.getRewriteable(), context, true);
assert searchContext.getSearchExecutionContext().isCacheable();
success = true;
} finally {
if (success == false) {
// we handle the case where `IndicesService#indexServiceSafe`or `IndexService#getShard`, or 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(searchContext);
}
}
return searchContext;
}
private void freeAllContextForIndex(Index index) {
assert index != null;
for (ReaderContext ctx : activeReaders.values()) {
if (index.equals(ctx.indexShard().shardId().getIndex())) {
freeReaderContext(ctx.id());
}
}
}
public boolean freeReaderContext(ShardSearchContextId contextId) {
if (sessionId.equals(contextId.getSessionId()) || contextId.getSessionId().isEmpty()) {
try (ReaderContext context = removeReaderContext(contextId.getId())) {
return context != null;
}
}
return false;
}
public void freeAllScrollContexts() {
for (ReaderContext readerContext : activeReaders.values()) {
if (readerContext.scrollContext() != null) {
freeReaderContext(readerContext.id());
}
}
}
private long getKeepAlive(ShardSearchRequest request) {
if (request.scroll() != null) {
return getScrollKeepAlive(request.scroll());
} else if (request.keepAlive() != null) {
checkKeepAliveLimit(request.keepAlive().millis());
return request.keepAlive().getMillis();
} else {
return request.readerId() == null ? defaultKeepAlive : -1;
}
}
private long getScrollKeepAlive(Scroll scroll) {
if (scroll != null && scroll.keepAlive() != null) {
checkKeepAliveLimit(scroll.keepAlive().millis());
return scroll.keepAlive().getMillis();
}
return defaultKeepAlive;
}
private void checkKeepAliveLimit(long keepAlive) {
if (keepAlive > maxKeepAlive) {
throw new IllegalArgumentException(
"Keep alive for request (" + TimeValue.timeValueMillis(keepAlive) + ") is too large. " +
"It must be less than (" + TimeValue.timeValueMillis(maxKeepAlive) + "). " +
"This limit can be set by changing the [" + MAX_KEEPALIVE_SETTING.getKey() + "] cluster level setting.");
}
}
private <T> ActionListener<T> wrapFailureListener(ActionListener<T> listener, ReaderContext context, Releasable releasable) {
return new ActionListener<T>() {
@Override
public void onResponse(T resp) {
Releasables.close(releasable);
listener.onResponse(resp);
}
@Override
public void onFailure(Exception exc) {
processFailure(context, exc);
Releasables.close(releasable);
listener.onFailure(exc);
}
};
}
private boolean isScrollContext(ReaderContext context) {
return context instanceof LegacyReaderContext && context.singleSession() == false;
}
private void processFailure(ReaderContext context, Exception exc) {
if (context.singleSession() || isScrollContext(context)) {
// we release the reader on failure if the request is a normal search or a scroll
freeReaderContext(context.id());
}
try {
if (Lucene.isCorruptionException(exc)) {
context.indexShard().failShard("search execution corruption failure", exc);
}
} catch (Exception inner) {
inner.addSuppressed(exc);
logger.warn("failed to process shard failure to (potentially) send back shard failure on corruption", inner);
}
}
private void parseSource(DefaultSearchContext context, SearchSourceBuilder source, boolean includeAggregations) {
// nothing to parse...
if (source == null) {
return;
}
SearchShardTarget shardTarget = context.shardTarget();
SearchExecutionContext searchExecutionContext = context.getSearchExecutionContext();
context.from(source.from());
context.size(source.size());
Map<String, InnerHitContextBuilder> innerHitBuilders = new HashMap<>();
if (source.query() != null) {
InnerHitContextBuilder.extractInnerHits(source.query(), innerHitBuilders);
context.parsedQuery(searchExecutionContext.toQuery(source.query()));
}
if (source.postFilter() != null) {
InnerHitContextBuilder.extractInnerHits(source.postFilter(), innerHitBuilders);
context.parsedPostFilter(searchExecutionContext.toQuery(source.postFilter()));
}
if (innerHitBuilders.size() > 0) {
for (Map.Entry<String, InnerHitContextBuilder> entry : innerHitBuilders.entrySet()) {
try {
entry.getValue().build(context, context.innerHits());
} catch (IOException e) {
throw new SearchException(shardTarget, "failed to build inner_hits", e);
}
}
}
if (source.sorts() != null) {
try {
Optional<SortAndFormats> optionalSort = SortBuilder.buildSort(source.sorts(), context.getSearchExecutionContext());
if (optionalSort.isPresent()) {
context.sort(optionalSort.get());
}
} catch (IOException e) {
throw new SearchException(shardTarget, "failed to create sort elements", e);
}
}
context.trackScores(source.trackScores());
if (source.trackTotalHitsUpTo() != null
&& source.trackTotalHitsUpTo() != SearchContext.TRACK_TOTAL_HITS_ACCURATE
&& context.scrollContext() != null) {
throw new SearchException(shardTarget, "disabling [track_total_hits] is not allowed in a scroll context");
}
if (source.trackTotalHitsUpTo() != null) {
context.trackTotalHitsUpTo(source.trackTotalHitsUpTo());
}
if (source.minScore() != null) {
context.minimumScore(source.minScore());
}
if (source.profile()) {
context.setProfilers(new Profilers(context.searcher()));
}
if (source.timeout() != null) {
context.timeout(source.timeout());
}
context.terminateAfter(source.terminateAfter());
if (source.aggregations() != null && includeAggregations) {
AggregationContext aggContext = new ProductionAggregationContext(
context.getSearchExecutionContext(),
bigArrays,
source.aggregations().bytesToPreallocate(),
/*
* The query on the search context right now doesn't include
* the filter for nested documents or slicing so we have to
* delay reading it until the aggs ask for it.
*/
() -> context.query() == null ? new MatchAllDocsQuery() : context.query(),
context.getProfilers() == null ? null : context.getProfilers().getAggregationProfiler(),
multiBucketConsumerService.create(),
() -> new SubSearchContext(context).parsedQuery(context.parsedQuery()).fetchFieldsContext(context.fetchFieldsContext()),
context.bitsetFilterCache(),
context.indexShard().shardId().hashCode(),
context::getRelativeTimeInMillis,
context::isCancelled
);
context.addReleasable(aggContext);
try {
AggregatorFactories factories = source.aggregations().build(aggContext, null);
context.aggregations(new SearchContextAggregations(factories));
} catch (IOException e) {
throw new AggregationInitializationException("Failed to create aggregators", e);
}
}
if (source.suggest() != null) {
try {
context.suggest(source.suggest().build(searchExecutionContext));
} catch (IOException e) {
throw new SearchException(shardTarget, "failed to create SuggestionSearchContext", e);
}
}
if (source.rescores() != null) {
try {
for (RescorerBuilder<?> rescore : source.rescores()) {
context.addRescore(rescore.buildContext(searchExecutionContext));
}
} catch (IOException e) {
throw new SearchException(shardTarget, "failed to create RescoreSearchContext", e);
}
}
if (source.explain() != null) {
context.explain(source.explain());
}
if (source.fetchSource() != null) {