From 987049e4d757e202fb25412fd2b349067287f7a3 Mon Sep 17 00:00:00 2001 From: Martin Gaievski Date: Thu, 2 May 2024 11:37:53 -0700 Subject: [PATCH] Refactor implementations of query phase searcher, add empty QueryCollectorContext (#13481) * Refactor implementations of query hpase searcher by adding overloaded searchWith method Signed-off-by: Martin Gaievski * Switched to Empty context add rescoring interface Signed-off-by: Martin Gaievski * Changed if by simple null check for querycontext argument Signed-off-by: Martin Gaievski * Added Override annotation for searchWith method Signed-off-by: Martin Gaievski * Remove old override method from Concurrent Query Phase Searcher Signed-off-by: Martin Gaievski --------- Signed-off-by: Martin Gaievski --- CHANGELOG.md | 1 + .../query/ConcurrentQueryPhaseSearcher.java | 26 +++++++++----- .../search/query/QueryCollectorContext.java | 23 ++++++++++++ .../opensearch/search/query/QueryPhase.java | 36 +++++++++++++++---- .../query/RescoringQueryCollectorContext.java | 27 ++++++++++++++ .../search/query/TopDocsCollectorContext.java | 2 +- 6 files changed, 100 insertions(+), 15 deletions(-) create mode 100644 server/src/main/java/org/opensearch/search/query/RescoringQueryCollectorContext.java diff --git a/CHANGELOG.md b/CHANGELOG.md index 5bff49af99473..ca2b174c98001 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -71,6 +71,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - [BWC and API enforcement] Reconsider the breaking changes check policy to detect breaking changes against released versions ([#13292](https://github.com/opensearch-project/OpenSearch/pull/13292)) - Switch to macos-13 runner for precommit and assemble github actions due to macos-latest is now arm64 ([#13412](https://github.com/opensearch-project/OpenSearch/pull/13412)) - [Revert] Prevent unnecessary fetch sub phase processor initialization during fetch phase execution ([#12503](https://github.com/opensearch-project/OpenSearch/pull/12503)) +- Refactor implementations of query phase searcher, allow QueryCollectorContext to have zero collectors ([#13481](https://github.com/opensearch-project/OpenSearch/pull/13481)) ### Deprecated diff --git a/server/src/main/java/org/opensearch/search/query/ConcurrentQueryPhaseSearcher.java b/server/src/main/java/org/opensearch/search/query/ConcurrentQueryPhaseSearcher.java index e22f766d3894c..771ac60dfb5e5 100644 --- a/server/src/main/java/org/opensearch/search/query/ConcurrentQueryPhaseSearcher.java +++ b/server/src/main/java/org/opensearch/search/query/ConcurrentQueryPhaseSearcher.java @@ -23,10 +23,9 @@ import java.io.IOException; import java.util.LinkedList; +import java.util.Objects; import java.util.concurrent.ExecutionException; -import static org.opensearch.search.query.TopDocsCollectorContext.createTopDocsCollectorContext; - /** * The implementation of the {@link QueryPhaseSearcher} which attempts to use concurrent * search of Apache Lucene segments if it has been enabled. @@ -46,10 +45,19 @@ protected boolean searchWithCollector( ContextIndexSearcher searcher, Query query, LinkedList collectors, + QueryCollectorContext queryCollectorContext, boolean hasFilterCollector, boolean hasTimeout ) throws IOException { - return searchWithCollectorManager(searchContext, searcher, query, collectors, hasFilterCollector, hasTimeout); + return searchWithCollectorManager( + searchContext, + searcher, + query, + collectors, + queryCollectorContext, + hasFilterCollector, + hasTimeout + ); } private static boolean searchWithCollectorManager( @@ -57,13 +65,12 @@ private static boolean searchWithCollectorManager( ContextIndexSearcher searcher, Query query, LinkedList collectorContexts, + QueryCollectorContext queryCollectorContext, 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 - collectorContexts.addFirst(topDocsFactory); + // add the passed collector, the first collector context in the chain + collectorContexts.addFirst(Objects.requireNonNull(queryCollectorContext)); final QuerySearchResult queryResult = searchContext.queryResult(); final CollectorManager collectorManager; @@ -95,7 +102,10 @@ private static boolean searchWithCollectorManager( queryResult.terminatedEarly(false); } - return topDocsFactory.shouldRescore(); + if (queryCollectorContext instanceof RescoringQueryCollectorContext) { + return ((RescoringQueryCollectorContext) queryCollectorContext).shouldRescore(); + } + return false; } @Override diff --git a/server/src/main/java/org/opensearch/search/query/QueryCollectorContext.java b/server/src/main/java/org/opensearch/search/query/QueryCollectorContext.java index 91762bee2ac08..08b048cf682bb 100644 --- a/server/src/main/java/org/opensearch/search/query/QueryCollectorContext.java +++ b/server/src/main/java/org/opensearch/search/query/QueryCollectorContext.java @@ -77,6 +77,29 @@ public ScoreMode scoreMode() { } }; + public static final QueryCollectorContext EMPTY_CONTEXT = new QueryCollectorContext("empty") { + + @Override + Collector create(Collector in) throws IOException { + return EMPTY_COLLECTOR; + } + + @Override + CollectorManager createManager(CollectorManager in) throws IOException { + return new CollectorManager() { + @Override + public Collector newCollector() throws IOException { + return EMPTY_COLLECTOR; + } + + @Override + public ReduceableSearchResult reduce(Collection collectors) throws IOException { + return result -> {}; + } + }; + } + }; + private String profilerName; QueryCollectorContext(String profilerName) { diff --git a/server/src/main/java/org/opensearch/search/query/QueryPhase.java b/server/src/main/java/org/opensearch/search/query/QueryPhase.java index 8f98f0d9efbd4..608649ad22b23 100644 --- a/server/src/main/java/org/opensearch/search/query/QueryPhase.java +++ b/server/src/main/java/org/opensearch/search/query/QueryPhase.java @@ -335,13 +335,12 @@ private static boolean searchWithCollector( ContextIndexSearcher searcher, Query query, LinkedList collectors, + QueryCollectorContext queryCollectorContext, 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); + // add passed collector, the first collector context in the chain + collectors.addFirst(Objects.requireNonNull(queryCollectorContext)); final Collector queryCollector; if (searchContext.getProfilers() != null) { @@ -370,7 +369,10 @@ private static boolean searchWithCollector( for (QueryCollectorContext ctx : collectors) { ctx.postProcess(queryResult); } - return topDocsFactory.shouldRescore(); + if (queryCollectorContext instanceof RescoringQueryCollectorContext) { + return ((RescoringQueryCollectorContext) queryCollectorContext).shouldRescore(); + } + return false; } /** @@ -440,7 +442,29 @@ protected boolean searchWithCollector( boolean hasFilterCollector, boolean hasTimeout ) throws IOException { - return QueryPhase.searchWithCollector(searchContext, searcher, query, collectors, hasFilterCollector, hasTimeout); + // create the top docs collector last when the other collectors are known + final TopDocsCollectorContext topDocsFactory = createTopDocsCollectorContext(searchContext, hasFilterCollector); + return searchWithCollector(searchContext, searcher, query, collectors, topDocsFactory, hasFilterCollector, hasTimeout); + } + + protected boolean searchWithCollector( + SearchContext searchContext, + ContextIndexSearcher searcher, + Query query, + LinkedList collectors, + QueryCollectorContext queryCollectorContext, + boolean hasFilterCollector, + boolean hasTimeout + ) throws IOException { + return QueryPhase.searchWithCollector( + searchContext, + searcher, + query, + collectors, + queryCollectorContext, + hasFilterCollector, + hasTimeout + ); } } } diff --git a/server/src/main/java/org/opensearch/search/query/RescoringQueryCollectorContext.java b/server/src/main/java/org/opensearch/search/query/RescoringQueryCollectorContext.java new file mode 100644 index 0000000000000..dc47ee80dba42 --- /dev/null +++ b/server/src/main/java/org/opensearch/search/query/RescoringQueryCollectorContext.java @@ -0,0 +1,27 @@ +/* + * 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. + */ + +package org.opensearch.search.query; + +import org.opensearch.common.annotation.PublicApi; + +/** + * Abstraction that allows indication of whether results should be rescored or not based on + * custom logic of exact {@link QueryCollectorContext} implementation. + * + * @opensearch.api + */ +@PublicApi(since = "2.15.0") +public interface RescoringQueryCollectorContext { + + /** + * Indicates if results from the query context should be rescored + * @return true if results must be rescored, false otherwise + */ + boolean shouldRescore(); +} diff --git a/server/src/main/java/org/opensearch/search/query/TopDocsCollectorContext.java b/server/src/main/java/org/opensearch/search/query/TopDocsCollectorContext.java index 65d3948c8401e..f780f6fe32af2 100644 --- a/server/src/main/java/org/opensearch/search/query/TopDocsCollectorContext.java +++ b/server/src/main/java/org/opensearch/search/query/TopDocsCollectorContext.java @@ -95,7 +95,7 @@ * * @opensearch.internal */ -public abstract class TopDocsCollectorContext extends QueryCollectorContext { +public abstract class TopDocsCollectorContext extends QueryCollectorContext implements RescoringQueryCollectorContext { protected final int numHits; TopDocsCollectorContext(String profilerName, int numHits) {