|
@@ -30,6 +30,8 @@ import org.elasticsearch.action.search.SearchType;
|
|
|
import org.elasticsearch.cluster.ClusterState;
|
|
|
import org.elasticsearch.cluster.service.ClusterService;
|
|
|
import org.elasticsearch.common.component.AbstractLifecycleComponent;
|
|
|
+import org.elasticsearch.common.io.stream.StreamInput;
|
|
|
+import org.elasticsearch.common.io.stream.StreamOutput;
|
|
|
import org.elasticsearch.common.lucene.Lucene;
|
|
|
import org.elasticsearch.common.settings.Setting;
|
|
|
import org.elasticsearch.common.settings.Setting.Property;
|
|
@@ -106,8 +108,10 @@ import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.Optional;
|
|
|
import java.util.concurrent.ExecutionException;
|
|
|
+import java.util.concurrent.Executor;
|
|
|
import java.util.concurrent.atomic.AtomicLong;
|
|
|
import java.util.function.LongSupplier;
|
|
|
+import java.util.function.Supplier;
|
|
|
|
|
|
import static org.elasticsearch.common.unit.TimeValue.timeValueHours;
|
|
|
import static org.elasticsearch.common.unit.TimeValue.timeValueMillis;
|
|
@@ -344,7 +348,21 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
|
|
|
});
|
|
|
}
|
|
|
|
|
|
- SearchPhaseResult executeQueryPhase(ShardSearchRequest request, SearchTask task) throws IOException {
|
|
|
+ private <T> void runAsync(long id, Supplier<T> executable, ActionListener<T> listener) {
|
|
|
+ getExecutor(id).execute(new AbstractRunnable() {
|
|
|
+ @Override
|
|
|
+ public void onFailure(Exception e) {
|
|
|
+ listener.onFailure(e);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ protected void doRun() {
|
|
|
+ listener.onResponse(executable.get());
|
|
|
+ }
|
|
|
+ });
|
|
|
+ }
|
|
|
+
|
|
|
+ private SearchPhaseResult executeQueryPhase(ShardSearchRequest request, SearchTask task) throws IOException {
|
|
|
final SearchContext context = createAndPutContext(request);
|
|
|
final SearchOperationListener operationListener = context.indexShard().getSearchOperationListener();
|
|
|
context.incRef();
|
|
@@ -405,59 +423,63 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
|
|
|
return new QueryFetchSearchResult(context.queryResult(), context.fetchResult());
|
|
|
}
|
|
|
|
|
|
- public ScrollQuerySearchResult executeQueryPhase(InternalScrollSearchRequest request, SearchTask task) {
|
|
|
- final SearchContext context = findContext(request.id(), request);
|
|
|
- SearchOperationListener operationListener = context.indexShard().getSearchOperationListener();
|
|
|
- context.incRef();
|
|
|
- try {
|
|
|
- context.setTask(task);
|
|
|
- operationListener.onPreQueryPhase(context);
|
|
|
- long time = System.nanoTime();
|
|
|
- contextProcessing(context);
|
|
|
- processScroll(request, context);
|
|
|
- queryPhase.execute(context);
|
|
|
- contextProcessedSuccessfully(context);
|
|
|
- operationListener.onQueryPhase(context, System.nanoTime() - time);
|
|
|
- return new ScrollQuerySearchResult(context.queryResult(), context.shardTarget());
|
|
|
- } catch (Exception e) {
|
|
|
- operationListener.onFailedQueryPhase(context);
|
|
|
- logger.trace("Query phase failed", e);
|
|
|
- processFailure(context, e);
|
|
|
- throw ExceptionsHelper.convertToRuntime(e);
|
|
|
- } finally {
|
|
|
- cleanContext(context);
|
|
|
- }
|
|
|
+ public void executeQueryPhase(InternalScrollSearchRequest request, SearchTask task, ActionListener<ScrollQuerySearchResult> listener) {
|
|
|
+ runAsync(request.id(), () -> {
|
|
|
+ final SearchContext context = findContext(request.id(), request);
|
|
|
+ SearchOperationListener operationListener = context.indexShard().getSearchOperationListener();
|
|
|
+ context.incRef();
|
|
|
+ try {
|
|
|
+ context.setTask(task);
|
|
|
+ operationListener.onPreQueryPhase(context);
|
|
|
+ long time = System.nanoTime();
|
|
|
+ contextProcessing(context);
|
|
|
+ processScroll(request, context);
|
|
|
+ queryPhase.execute(context);
|
|
|
+ contextProcessedSuccessfully(context);
|
|
|
+ operationListener.onQueryPhase(context, System.nanoTime() - time);
|
|
|
+ return new ScrollQuerySearchResult(context.queryResult(), context.shardTarget());
|
|
|
+ } catch (Exception e) {
|
|
|
+ operationListener.onFailedQueryPhase(context);
|
|
|
+ logger.trace("Query phase failed", e);
|
|
|
+ processFailure(context, e);
|
|
|
+ throw ExceptionsHelper.convertToRuntime(e);
|
|
|
+ } finally {
|
|
|
+ cleanContext(context);
|
|
|
+ }
|
|
|
+ }, listener);
|
|
|
}
|
|
|
|
|
|
- public QuerySearchResult executeQueryPhase(QuerySearchRequest request, SearchTask task) {
|
|
|
- final SearchContext context = findContext(request.id(), request);
|
|
|
- context.setTask(task);
|
|
|
- IndexShard indexShard = context.indexShard();
|
|
|
- SearchOperationListener operationListener = indexShard.getSearchOperationListener();
|
|
|
- context.incRef();
|
|
|
- try {
|
|
|
- contextProcessing(context);
|
|
|
- context.searcher().setAggregatedDfs(request.dfs());
|
|
|
+ public void executeQueryPhase(QuerySearchRequest request, SearchTask task, ActionListener<QuerySearchResult> listener) {
|
|
|
+ runAsync(request.id(), () -> {
|
|
|
+ final SearchContext context = findContext(request.id(), request);
|
|
|
+ context.setTask(task);
|
|
|
+ IndexShard indexShard = context.indexShard();
|
|
|
+ SearchOperationListener operationListener = indexShard.getSearchOperationListener();
|
|
|
+ context.incRef();
|
|
|
+ try {
|
|
|
+ contextProcessing(context);
|
|
|
+ context.searcher().setAggregatedDfs(request.dfs());
|
|
|
|
|
|
- operationListener.onPreQueryPhase(context);
|
|
|
- long time = System.nanoTime();
|
|
|
- queryPhase.execute(context);
|
|
|
- if (context.queryResult().hasSearchContext() == false && context.scrollContext() == null) {
|
|
|
- // no hits, we can release the context since there will be no fetch phase
|
|
|
- freeContext(context.id());
|
|
|
- } else {
|
|
|
- contextProcessedSuccessfully(context);
|
|
|
+ operationListener.onPreQueryPhase(context);
|
|
|
+ long time = System.nanoTime();
|
|
|
+ queryPhase.execute(context);
|
|
|
+ if (context.queryResult().hasSearchContext() == false && context.scrollContext() == null) {
|
|
|
+ // no hits, we can release the context since there will be no fetch phase
|
|
|
+ freeContext(context.id());
|
|
|
+ } else {
|
|
|
+ contextProcessedSuccessfully(context);
|
|
|
+ }
|
|
|
+ operationListener.onQueryPhase(context, System.nanoTime() - time);
|
|
|
+ return context.queryResult();
|
|
|
+ } catch (Exception e) {
|
|
|
+ operationListener.onFailedQueryPhase(context);
|
|
|
+ logger.trace("Query phase failed", e);
|
|
|
+ processFailure(context, e);
|
|
|
+ throw ExceptionsHelper.convertToRuntime(e);
|
|
|
+ } finally {
|
|
|
+ cleanContext(context);
|
|
|
}
|
|
|
- operationListener.onQueryPhase(context, System.nanoTime() - time);
|
|
|
- return context.queryResult();
|
|
|
- } catch (Exception e) {
|
|
|
- operationListener.onFailedQueryPhase(context);
|
|
|
- logger.trace("Query phase failed", e);
|
|
|
- processFailure(context, e);
|
|
|
- throw ExceptionsHelper.convertToRuntime(e);
|
|
|
- } finally {
|
|
|
- cleanContext(context);
|
|
|
- }
|
|
|
+ }, listener);
|
|
|
}
|
|
|
|
|
|
private boolean fetchPhaseShouldFreeContext(SearchContext context) {
|
|
@@ -470,66 +492,83 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- public ScrollQueryFetchSearchResult executeFetchPhase(InternalScrollSearchRequest request, SearchTask task) {
|
|
|
- final SearchContext context = findContext(request.id(), request);
|
|
|
- context.incRef();
|
|
|
- try {
|
|
|
- context.setTask(task);
|
|
|
- contextProcessing(context);
|
|
|
- SearchOperationListener operationListener = context.indexShard().getSearchOperationListener();
|
|
|
- processScroll(request, context);
|
|
|
- operationListener.onPreQueryPhase(context);
|
|
|
- final long time = System.nanoTime();
|
|
|
+ final Executor getExecutor(long id) {
|
|
|
+ SearchContext context = activeContexts.get(id);
|
|
|
+ if (context == null) {
|
|
|
+ throw new SearchContextMissingException(id);
|
|
|
+ }
|
|
|
+ return getExecutor(context.indexShard());
|
|
|
+ }
|
|
|
+
|
|
|
+ private Executor getExecutor(IndexShard indexShard) {
|
|
|
+ assert indexShard != null;
|
|
|
+ return threadPool.executor(indexShard.indexSettings().isSearchThrottled() ? Names.SEARCH_THROTTLED : Names.SEARCH);
|
|
|
+ }
|
|
|
+
|
|
|
+ public void executeFetchPhase(InternalScrollSearchRequest request, SearchTask task,
|
|
|
+ ActionListener<ScrollQueryFetchSearchResult> listener) {
|
|
|
+ runAsync(request.id(), () -> {
|
|
|
+ final SearchContext context = findContext(request.id(), request);
|
|
|
+ context.incRef();
|
|
|
try {
|
|
|
- queryPhase.execute(context);
|
|
|
+ context.setTask(task);
|
|
|
+ contextProcessing(context);
|
|
|
+ SearchOperationListener operationListener = context.indexShard().getSearchOperationListener();
|
|
|
+ processScroll(request, context);
|
|
|
+ operationListener.onPreQueryPhase(context);
|
|
|
+ final long time = System.nanoTime();
|
|
|
+ try {
|
|
|
+ queryPhase.execute(context);
|
|
|
+ } catch (Exception e) {
|
|
|
+ operationListener.onFailedQueryPhase(context);
|
|
|
+ throw ExceptionsHelper.convertToRuntime(e);
|
|
|
+ }
|
|
|
+ long afterQueryTime = System.nanoTime();
|
|
|
+ operationListener.onQueryPhase(context, afterQueryTime - time);
|
|
|
+ QueryFetchSearchResult fetchSearchResult = executeFetchPhase(context, operationListener, afterQueryTime);
|
|
|
+ return new ScrollQueryFetchSearchResult(fetchSearchResult,
|
|
|
+ context.shardTarget());
|
|
|
} catch (Exception e) {
|
|
|
- operationListener.onFailedQueryPhase(context);
|
|
|
+ logger.trace("Fetch phase failed", e);
|
|
|
+ processFailure(context, e);
|
|
|
throw ExceptionsHelper.convertToRuntime(e);
|
|
|
+ } finally {
|
|
|
+ cleanContext(context);
|
|
|
}
|
|
|
- long afterQueryTime = System.nanoTime();
|
|
|
- operationListener.onQueryPhase(context, afterQueryTime - time);
|
|
|
- QueryFetchSearchResult fetchSearchResult = executeFetchPhase(context, operationListener, afterQueryTime);
|
|
|
-
|
|
|
- return new ScrollQueryFetchSearchResult(fetchSearchResult,
|
|
|
- context.shardTarget());
|
|
|
- } catch (Exception e) {
|
|
|
- logger.trace("Fetch phase failed", e);
|
|
|
- processFailure(context, e);
|
|
|
- throw ExceptionsHelper.convertToRuntime(e);
|
|
|
- } finally {
|
|
|
- cleanContext(context);
|
|
|
- }
|
|
|
+ }, listener);
|
|
|
}
|
|
|
|
|
|
- public FetchSearchResult executeFetchPhase(ShardFetchRequest request, SearchTask task) {
|
|
|
- final SearchContext context = findContext(request.id(), request);
|
|
|
- final SearchOperationListener operationListener = context.indexShard().getSearchOperationListener();
|
|
|
- context.incRef();
|
|
|
- try {
|
|
|
- context.setTask(task);
|
|
|
- contextProcessing(context);
|
|
|
- if (request.lastEmittedDoc() != null) {
|
|
|
- context.scrollContext().lastEmittedDoc = request.lastEmittedDoc();
|
|
|
- }
|
|
|
- context.docIdsToLoad(request.docIds(), 0, request.docIdsSize());
|
|
|
- operationListener.onPreFetchPhase(context);
|
|
|
- long time = System.nanoTime();
|
|
|
- fetchPhase.execute(context);
|
|
|
- if (fetchPhaseShouldFreeContext(context)) {
|
|
|
- freeContext(request.id());
|
|
|
- } else {
|
|
|
- contextProcessedSuccessfully(context);
|
|
|
+ public void executeFetchPhase(ShardFetchRequest request, SearchTask task, ActionListener<FetchSearchResult> listener) {
|
|
|
+ runAsync(request.id(), () -> {
|
|
|
+ final SearchContext context = findContext(request.id(), request);
|
|
|
+ final SearchOperationListener operationListener = context.indexShard().getSearchOperationListener();
|
|
|
+ context.incRef();
|
|
|
+ try {
|
|
|
+ context.setTask(task);
|
|
|
+ contextProcessing(context);
|
|
|
+ if (request.lastEmittedDoc() != null) {
|
|
|
+ context.scrollContext().lastEmittedDoc = request.lastEmittedDoc();
|
|
|
+ }
|
|
|
+ context.docIdsToLoad(request.docIds(), 0, request.docIdsSize());
|
|
|
+ operationListener.onPreFetchPhase(context);
|
|
|
+ long time = System.nanoTime();
|
|
|
+ fetchPhase.execute(context);
|
|
|
+ if (fetchPhaseShouldFreeContext(context)) {
|
|
|
+ freeContext(request.id());
|
|
|
+ } else {
|
|
|
+ contextProcessedSuccessfully(context);
|
|
|
+ }
|
|
|
+ operationListener.onFetchPhase(context, System.nanoTime() - time);
|
|
|
+ return context.fetchResult();
|
|
|
+ } catch (Exception e) {
|
|
|
+ operationListener.onFailedFetchPhase(context);
|
|
|
+ logger.trace("Fetch phase failed", e);
|
|
|
+ processFailure(context, e);
|
|
|
+ throw ExceptionsHelper.convertToRuntime(e);
|
|
|
+ } finally {
|
|
|
+ cleanContext(context);
|
|
|
}
|
|
|
- operationListener.onFetchPhase(context, System.nanoTime() - time);
|
|
|
- return context.fetchResult();
|
|
|
- } catch (Exception e) {
|
|
|
- operationListener.onFailedFetchPhase(context);
|
|
|
- logger.trace("Fetch phase failed", e);
|
|
|
- processFailure(context, e);
|
|
|
- throw ExceptionsHelper.convertToRuntime(e);
|
|
|
- } finally {
|
|
|
- cleanContext(context);
|
|
|
- }
|
|
|
+ }, listener);
|
|
|
}
|
|
|
|
|
|
private SearchContext findContext(long id, TransportRequest request) throws SearchContextMissingException {
|
|
@@ -985,6 +1024,15 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+
|
|
|
+ public void canMatch(ShardSearchRequest request, ActionListener<CanMatchResponse> listener) {
|
|
|
+ try {
|
|
|
+ listener.onResponse(new CanMatchResponse(canMatch(request)));
|
|
|
+ } catch (IOException e) {
|
|
|
+ listener.onFailure(e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Returns true iff the given search source builder can be early terminated by rewriting to a match none query. Or in other words
|
|
|
* if the execution of a the search request can be early terminated without executing it. This is for instance not possible if
|
|
@@ -1009,31 +1057,27 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
|
|
|
* The action listener is guaranteed to be executed on the search thread-pool
|
|
|
*/
|
|
|
private void rewriteShardRequest(ShardSearchRequest request, ActionListener<ShardSearchRequest> listener) {
|
|
|
+ IndexShard shard = indicesService.indexServiceSafe(request.shardId().getIndex()).getShard(request.shardId().id());
|
|
|
+ Executor executor = getExecutor(shard);
|
|
|
ActionListener<Rewriteable> actionListener = ActionListener.wrap(r ->
|
|
|
- threadPool.executor(Names.SEARCH).execute(new AbstractRunnable() {
|
|
|
- @Override
|
|
|
- public void onFailure(Exception e) {
|
|
|
- listener.onFailure(e);
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- protected void doRun() throws Exception {
|
|
|
- listener.onResponse(request);
|
|
|
- }
|
|
|
- }), listener::onFailure);
|
|
|
- IndexShard shardOrNull = indicesService.getShardOrNull(request.shardId());
|
|
|
- if (shardOrNull != null) {
|
|
|
// now we need to check if there is a pending refresh and register
|
|
|
- ActionListener<Rewriteable> finalListener = actionListener;
|
|
|
- actionListener = ActionListener.wrap(r ->
|
|
|
- shardOrNull.awaitShardSearchActive(b -> finalListener.onResponse(r)), finalListener::onFailure);
|
|
|
- }
|
|
|
+ shard.awaitShardSearchActive(b ->
|
|
|
+ executor.execute(new AbstractRunnable() {
|
|
|
+ @Override
|
|
|
+ public void onFailure(Exception e) {
|
|
|
+ listener.onFailure(e);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ protected void doRun() {
|
|
|
+ listener.onResponse(request);
|
|
|
+ }
|
|
|
+ })
|
|
|
+ ), 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);
|
|
|
-
|
|
|
-
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -1050,4 +1094,31 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
|
|
|
public InternalAggregation.ReduceContext createReduceContext(boolean finalReduce) {
|
|
|
return new InternalAggregation.ReduceContext(bigArrays, scriptService, multiBucketConsumerService.create(), finalReduce);
|
|
|
}
|
|
|
+
|
|
|
+ public static final class CanMatchResponse extends SearchPhaseResult {
|
|
|
+ private boolean canMatch;
|
|
|
+
|
|
|
+ public CanMatchResponse() {
|
|
|
+ }
|
|
|
+
|
|
|
+ public CanMatchResponse(boolean canMatch) {
|
|
|
+ this.canMatch = canMatch;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void readFrom(StreamInput in) throws IOException {
|
|
|
+ super.readFrom(in);
|
|
|
+ canMatch = in.readBoolean();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void writeTo(StreamOutput out) throws IOException {
|
|
|
+ super.writeTo(out);
|
|
|
+ out.writeBoolean(canMatch);
|
|
|
+ }
|
|
|
+
|
|
|
+ public boolean canMatch() {
|
|
|
+ return canMatch;
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|