This is an automated email from the ASF dual-hosted git repository.
dsmiley pushed a commit to branch branch_9x
in repository https://gitbox.apache.org/repos/asf/solr.git
The following commit(s) were added to refs/heads/branch_9x by this push:
new 400cbb56cd3 SOLR-17319 : New Combined Query / hybrid search (RRF)
(#3418)
400cbb56cd3 is described below
commit 400cbb56cd3fc0eb384f7de944ccdf6b4699ea2d
Author: Sonu Sharma <[email protected]>
AuthorDate: Fri Dec 19 22:16:37 2025 +0530
SOLR-17319 : New Combined Query / hybrid search (RRF) (#3418)
New CombinedQuerySearchHandler etc. for implementing hybrid search with
reciprocal rank fusion (RRF).
See "JSON Combined Query DSL" in ref guide, and params prefixed with
"combiner".
QueryComponent: refactorings to enable a subclass to customize merging
shard results.
---------
Co-authored-by: Sonu Sharma <[email protected]>
Co-authored-by: Christine Poerschke <[email protected]>
Co-authored-by: David Smiley <[email protected]>
(cherry picked from commit 42cd889e8f0ace7ace2002bef2c98566b772f0f5)
---
.../SOLR-17319-added_combined_query_RRF.yml | 8 +
.../handler/component/CombinedQueryComponent.java | 622 +++++++++++++++++++++
.../component/CombinedQueryResponseBuilder.java | 55 ++
.../component/CombinedQuerySearchHandler.java | 64 +++
.../solr/handler/component/QueryComponent.java | 144 +++--
.../combine/QueryAndResponseCombiner.java | 130 +++++
.../component/combine/ReciprocalRankFusion.java | 167 ++++++
.../handler/component/combine/package-info.java | 22 +
.../collection1/conf/solrconfig-combined-query.xml | 168 ++++++
.../component/CombinedQueryComponentTest.java | 250 +++++++++
.../component/CombinedQuerySearchHandlerTest.java | 76 +++
.../component/CombinedQuerySolrCloudTest.java | 284 ++++++++++
.../DistributedCombinedQueryComponentTest.java | 293 ++++++++++
.../combine/QueryAndResponseCombinerTest.java | 61 ++
.../combine/ReciprocalRankFusionTest.java | 109 ++++
.../handler/component/combine/TestCombiner.java | 64 +++
.../query-guide/pages/json-combined-query-dsl.adoc | 107 ++++
.../modules/query-guide/querying-nav.adoc | 1 +
.../apache/solr/common/params/CombinerParams.java | 35 ++
19 files changed, 2608 insertions(+), 52 deletions(-)
diff --git a/changelog/unreleased/SOLR-17319-added_combined_query_RRF.yml
b/changelog/unreleased/SOLR-17319-added_combined_query_RRF.yml
new file mode 100644
index 00000000000..17d64bf0233
--- /dev/null
+++ b/changelog/unreleased/SOLR-17319-added_combined_query_RRF.yml
@@ -0,0 +1,8 @@
+title: New CombinedQuerySearchHandler etc. for implementing hybrid search with
reciprocal rank fusion (RRF).
+type: added
+authors:
+ - name: Sonu Sharma
+ - name: David Smiley
+links:
+ - name: SOLR-17319
+ url: https://issues.apache.org/jira/browse/SOLR-17319
diff --git
a/solr/core/src/java/org/apache/solr/handler/component/CombinedQueryComponent.java
b/solr/core/src/java/org/apache/solr/handler/component/CombinedQueryComponent.java
new file mode 100644
index 00000000000..1cf109dd81c
--- /dev/null
+++
b/solr/core/src/java/org/apache/solr/handler/component/CombinedQueryComponent.java
@@ -0,0 +1,622 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.handler.component;
+
+import static java.lang.Math.max;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.CombinerParams;
+import org.apache.solr.common.params.CursorMarkParams;
+import org.apache.solr.common.params.GroupParams;
+import org.apache.solr.common.params.ShardParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.handler.component.combine.QueryAndResponseCombiner;
+import org.apache.solr.handler.component.combine.ReciprocalRankFusion;
+import org.apache.solr.response.BasicResultContext;
+import org.apache.solr.response.ResultContext;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.search.DocListAndSet;
+import org.apache.solr.search.QueryResult;
+import org.apache.solr.search.SolrReturnFields;
+import org.apache.solr.search.SortSpec;
+import org.apache.solr.util.SolrResponseUtil;
+import org.apache.solr.util.plugin.SolrCoreAware;
+
+/**
+ * Extends QueryComponent with support for executing multiple queries and
combining their results,
+ * like hybrid search with RFF. Does nothing special if not activated.
+ *
+ * <p>When activated, this component only works in distributed search /
coordinator mode.
+ *
+ * @see CombinedQuerySearchHandler
+ * @see CombinerParams
+ * @see QueryAndResponseCombiner
+ */
+public class CombinedQueryComponent extends QueryComponent implements
SolrCoreAware {
+
+ public static final String COMPONENT_NAME = "combined_query";
+ protected NamedList<?> initParams;
+ private final Map<String, QueryAndResponseCombiner> combiners = new
HashMap<>();
+ private int maxCombinerQueries;
+ private static final String RESPONSE_PER_QUERY_KEY = "response_per_query";
+
+ @Override
+ public String getDescription() {
+ return "Combined Query Component to support multiple query execution like
RRF";
+ }
+
+ @Override
+ public void init(NamedList<?> args) {
+ super.init(args);
+ this.initParams = args;
+ this.maxCombinerQueries = CombinerParams.DEFAULT_MAX_COMBINER_QUERIES;
+ }
+
+ @Override
+ public void inform(SolrCore core) {
+ for (Map.Entry<String, ?> initEntry : initParams) {
+ if ("combiners".equals(initEntry.getKey()) && initEntry.getValue()
instanceof NamedList<?>) {
+ var all = (NamedList<?>) initEntry.getValue();
+ for (int i = 0; i < all.size(); i++) {
+ String name = all.getName(i);
+ NamedList<?> combinerConfig = (NamedList<?>) all.getVal(i);
+ String className = (String) combinerConfig.get("class");
+ QueryAndResponseCombiner combiner =
+ core.getResourceLoader().newInstance(className,
QueryAndResponseCombiner.class);
+ combiner.init(combinerConfig);
+ combiners.compute(
+ name,
+ (k, existingCombiner) -> {
+ if (existingCombiner == null) {
+ return combiner;
+ }
+ throw new SolrException(
+ SolrException.ErrorCode.BAD_REQUEST,
+ "Found more than one combiner with same name");
+ });
+ }
+ }
+ }
+ Object maxQueries = initParams.get("maxCombinerQueries");
+ if (maxQueries != null) {
+ this.maxCombinerQueries = Integer.parseInt(maxQueries.toString());
+ }
+ combiners.computeIfAbsent(
+ CombinerParams.RECIPROCAL_RANK_FUSION,
+ key -> {
+ ReciprocalRankFusion reciprocalRankFusion = new
ReciprocalRankFusion();
+ reciprocalRankFusion.init(initParams);
+ return reciprocalRankFusion;
+ });
+ }
+
+ @Override
+ public void prepare(ResponseBuilder rb) throws IOException {
+ if (rb instanceof CombinedQueryResponseBuilder) { // see
CombinedQuerySearchHandler
+ var crb = (CombinedQueryResponseBuilder) rb;
+ SolrParams params = crb.req.getParams();
+ if (params.get(CursorMarkParams.CURSOR_MARK_PARAM) != null
+ || params.getBool(GroupParams.GROUP, false)) {
+ throw new SolrException(
+ SolrException.ErrorCode.BAD_REQUEST, "Unsupported functionality
for Combined Queries.");
+ }
+ String[] queriesToCombineKeys =
params.getParams(CombinerParams.COMBINER_QUERY);
+ if (queriesToCombineKeys.length > maxCombinerQueries) {
+ throw new SolrException(
+ SolrException.ErrorCode.BAD_REQUEST,
+ "Too many queries to combine: limit is " + maxCombinerQueries);
+ }
+ for (String queryKey : queriesToCombineKeys) {
+ final var unparsedQuery = params.get(queryKey);
+ ResponseBuilder rbNew = new ResponseBuilder(rb.req, new
SolrQueryResponse(), rb.components);
+ rbNew.setQueryString(unparsedQuery);
+ super.prepare(rbNew);
+ crb.setFilters(rbNew.getFilters());
+ crb.responseBuilders.add(rbNew);
+ }
+ }
+ super.prepare(rb);
+ }
+
+ /**
+ * Overrides the process method to handle CombinedQueryResponseBuilder
instances. This method
+ * processes the responses from multiple queries, combines them using the
specified
+ * QueryAndResponseCombiner strategy, and sets the appropriate results and
metadata in the
+ * CombinedQueryResponseBuilder.
+ *
+ * @param rb the ResponseBuilder object to process
+ * @throws IOException if an I/O error occurs during processing
+ */
+ @Override
+ @SuppressWarnings("unchecked")
+ public void process(ResponseBuilder rb) throws IOException {
+ if (rb instanceof CombinedQueryResponseBuilder) { // see
CombinedQuerySearchHandler
+ var crb = (CombinedQueryResponseBuilder) rb;
+ boolean partialResults = false;
+ boolean segmentTerminatedEarly = false;
+ Boolean setMaxHitsTerminatedEarly = null;
+ List<QueryResult> queryResults = new ArrayList<>();
+ int rbIndex = 0;
+ boolean shouldReturn = false;
+ // TODO: to be parallelized
+ for (ResponseBuilder thisRb : crb.responseBuilders) {
+ // Just a placeholder for future implementation for Cursors
+ thisRb.setCursorMark(crb.getCursorMark());
+ super.process(thisRb);
+ int purpose =
+ thisRb
+ .req
+ .getParams()
+ .getInt(ShardParams.SHARDS_PURPOSE,
ShardRequest.PURPOSE_GET_TOP_IDS);
+ if ((purpose & ShardRequest.PURPOSE_GET_TERM_STATS) != 0) {
+ shouldReturn = true;
+ continue;
+ }
+ DocListAndSet docListAndSet = thisRb.getResults();
+ QueryResult queryResult = new QueryResult();
+ queryResult.setDocListAndSet(docListAndSet);
+ queryResults.add(queryResult);
+ partialResults |= queryResult.isPartialResults();
+ if (queryResult.getSegmentTerminatedEarly() != null) {
+ segmentTerminatedEarly |= queryResult.getSegmentTerminatedEarly();
+ }
+ if (queryResult.getMaxHitsTerminatedEarly() != null) {
+ if (setMaxHitsTerminatedEarly == null) {
+ setMaxHitsTerminatedEarly =
queryResult.getMaxHitsTerminatedEarly();
+ }
+ setMaxHitsTerminatedEarly |= queryResult.getMaxHitsTerminatedEarly();
+ }
+ doFieldSortValues(thisRb, crb.req.getSearcher());
+ NamedList<Object[]> sortValues =
+ (NamedList<Object[]>) thisRb.rsp.getValues().get("sort_values");
+ crb.rsp.add(String.format(Locale.ROOT, "sort_values_%s", rbIndex),
sortValues);
+ ResultContext ctx = new BasicResultContext(thisRb);
+ if (crb.rsp.getValues().get(RESPONSE_PER_QUERY_KEY) == null) {
+ crb.rsp.add(RESPONSE_PER_QUERY_KEY, new ArrayList<>(List.of(ctx)));
+ } else {
+ ((List<ResultContext>)
crb.rsp.getValues().get(RESPONSE_PER_QUERY_KEY)).add(ctx);
+ }
+ rbIndex++;
+ }
+ if (shouldReturn) {
+ return;
+ }
+ prepareCombinedResponseBuilder(
+ crb, queryResults, partialResults, segmentTerminatedEarly,
setMaxHitsTerminatedEarly);
+ if (crb.mergeFieldHandler != null) {
+ crb.mergeFieldHandler.handleMergeFields(crb, crb.req.getSearcher());
+ } else {
+ doFieldSortValues(rb, crb.req.getSearcher());
+ }
+ doPrefetch(crb);
+ } else {
+ super.process(rb);
+ }
+ }
+
+ private void prepareCombinedResponseBuilder(
+ CombinedQueryResponseBuilder crb,
+ List<QueryResult> queryResults,
+ boolean partialResults,
+ boolean segmentTerminatedEarly,
+ Boolean setMaxHitsTerminatedEarly) {
+ QueryResult combinedQueryResult =
QueryAndResponseCombiner.simpleCombine(queryResults);
+ combinedQueryResult.setPartialResults(partialResults);
+ combinedQueryResult.setSegmentTerminatedEarly(segmentTerminatedEarly);
+ combinedQueryResult.setMaxHitsTerminatedEarly(setMaxHitsTerminatedEarly);
+ crb.setResult(combinedQueryResult);
+ ResultContext ctx = new BasicResultContext(crb);
+ crb.rsp.addResponse(ctx);
+ crb.rsp.addToLog(
+ "hits",
+ crb.getResults() == null || crb.getResults().docList == null
+ ? 0
+ : crb.getResults().docList.matches());
+ if (!crb.req.getParams().getBool(ShardParams.IS_SHARD, false)) {
+ // for non-distributed request and future cursor improvement
+ if (null != crb.getNextCursorMark()) {
+ crb.rsp.add(
+ CursorMarkParams.CURSOR_MARK_NEXT,
+
crb.responseBuilders.get(0).getNextCursorMark().getSerializedTotem());
+ }
+ }
+ }
+
+ @Override
+ protected void mergeIds(ResponseBuilder rb, ShardRequest sreq) {
+ SortSpec ss = rb.getSortSpec();
+ Sort sort = ss.getSort();
+
+ SortField[] sortFields;
+ if (sort != null) sortFields = sort.getSort();
+ else {
+ sortFields = new SortField[] {SortField.FIELD_SCORE};
+ }
+
+ Set<String> scoreDependentFields;
+ if ((sreq.purpose & ShardRequest.PURPOSE_GET_FIELDS) == 0) {
+ scoreDependentFields =
+
rb.rsp.getReturnFields().getScoreDependentReturnFields().keySet().stream()
+ .filter(field -> !field.equals(SolrReturnFields.SCORE))
+ .collect(Collectors.toSet());
+ } else {
+ scoreDependentFields = Collections.emptySet();
+ }
+
+ IndexSchema schema = rb.req.getSchema();
+ SchemaField uniqueKeyField = schema.getUniqueKeyField();
+
+ NamedList<Object> shardInfo = null;
+ if (rb.req.getParams().getBool(ShardParams.SHARDS_INFO, false)) {
+ shardInfo = new SimpleOrderedMap<>();
+ rb.rsp.getValues().add(ShardParams.SHARDS_INFO, shardInfo);
+ }
+
+ long numFound = 0;
+ boolean hitCountIsExact = true;
+ boolean thereArePartialResults = false;
+ Boolean segmentTerminatedEarly = null;
+ boolean maxHitsTerminatedEarly = false;
+ long approximateTotalHits = 0;
+ Map<String, List<ShardDoc>> shardDocMap = new HashMap<>();
+ String[] queriesToCombineKeys =
rb.req.getParams().getParams(CombinerParams.COMBINER_QUERY);
+ // TODO: to be parallelized outer loop
+ for (int queryIndex = 0; queryIndex < queriesToCombineKeys.length;
queryIndex++) {
+ int failedShardCount = 0;
+ long queryNumFound = 0;
+ long queryApproximateTotalHits = 0;
+ final ShardDocQueue queuePerQuery =
+ newShardDocQueue(rb.req.getSearcher(), sortFields, ss.getOffset() +
ss.getCount());
+ for (ShardResponse srsp : sreq.responses) {
+ SolrDocumentList docs = null;
+ NamedList<?> responseHeader;
+
+ Object subsection =
+ SolrResponseUtil.getSubsectionFromShardResponse(
+ rb, srsp, RESPONSE_PER_QUERY_KEY, false);
+ if (subsection instanceof List<?>) {
+ List<?> docList = (List<?>) subsection;
+ Object dl = docList.get(queryIndex);
+ if (dl instanceof SolrDocumentList) {
+ docs = Objects.requireNonNull((SolrDocumentList) dl);
+ queryNumFound += docs.getNumFound();
+ hitCountIsExact = hitCountIsExact &&
Boolean.FALSE.equals(docs.getNumFoundExact());
+ }
+ }
+ failedShardCount +=
+ addShardInfo(
+ shardInfo, failedShardCount, srsp, rb,
queriesToCombineKeys[queryIndex], docs);
+ if (srsp.getException() != null) {
+ thereArePartialResults = true;
+ continue;
+ }
+
+ responseHeader =
+ Objects.requireNonNull(
+ (NamedList<?>)
+ SolrResponseUtil.getSubsectionFromShardResponse(
+ rb, srsp, "responseHeader", false));
+
+ final boolean thisResponseIsPartial;
+ thisResponseIsPartial =
+ Boolean.TRUE.equals(
+ responseHeader.getBooleanArg(
+ SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY));
+ thereArePartialResults |= thisResponseIsPartial;
+
+ if (!Boolean.TRUE.equals(segmentTerminatedEarly)) {
+ final Object ste =
+
responseHeader.get(SolrQueryResponse.RESPONSE_HEADER_SEGMENT_TERMINATED_EARLY_KEY);
+ if (ste instanceof Boolean) {
+ segmentTerminatedEarly = (Boolean) ste;
+ }
+ }
+
+ if (!maxHitsTerminatedEarly
+ && Boolean.TRUE.equals(
+ responseHeader.get(
+
SolrQueryResponse.RESPONSE_HEADER_MAX_HITS_TERMINATED_EARLY_KEY))) {
+ maxHitsTerminatedEarly = true;
+ }
+
+ Object ath =
+
responseHeader.get(SolrQueryResponse.RESPONSE_HEADER_APPROXIMATE_TOTAL_HITS_KEY);
+ if (ath == null) {
+ queryApproximateTotalHits += queryNumFound;
+ } else {
+ queryApproximateTotalHits += ((Number) ath).longValue();
+ }
+
+ @SuppressWarnings("unchecked")
+ NamedList<List<Object>> sortFieldValues =
+ (NamedList<List<Object>>)
+ SolrResponseUtil.getSubsectionFromShardResponse(
+ rb, srsp, String.format(Locale.ROOT, "sort_values_%s",
queryIndex), true);
+ if (null == sortFieldValues) {
+ sortFieldValues = new NamedList<>();
+ }
+
+ boolean needsUnmarshalling = ss.includesNonScoreOrDocField();
+ if (thisResponseIsPartial && sortFieldValues.size() == 0 &&
needsUnmarshalling) {
+ continue;
+ }
+ NamedList<List<Object>> unmarshalledSortFieldValues =
+ needsUnmarshalling
+ ? unmarshalSortValues(ss, sortFieldValues, schema)
+ : new NamedList<>();
+ // go through every doc in this response, construct a ShardDoc, and
+ // put it in the uniqueDoc to dedup
+ for (int i = 0; i < docs.size(); i++) {
+ SolrDocument doc = docs.get(i);
+ Object id = doc.getFieldValue(uniqueKeyField.getName());
+ ShardDoc shardDoc = new ShardDoc();
+ shardDoc.id = id;
+ shardDoc.orderInShard = i;
+ shardDoc.shard = srsp.getShard();
+ Object scoreObj = doc.getFieldValue(SolrReturnFields.SCORE);
+ if (scoreObj != null) {
+ if (scoreObj instanceof String) {
+ String scoreStr = (String) scoreObj;
+ shardDoc.score = Float.parseFloat(scoreStr);
+ } else {
+ shardDoc.score = ((Number) scoreObj).floatValue();
+ }
+ }
+ if (!scoreDependentFields.isEmpty()) {
+ shardDoc.scoreDependentFields =
doc.getSubsetOfFields(scoreDependentFields);
+ }
+ shardDoc.sortFieldValues = unmarshalledSortFieldValues;
+ if (!queuePerQuery.push(shardDoc)) {
+ numFound--;
+ }
+ } // end for-each-doc-in-response
+ } // end for-each-response
+ List<ShardDoc> shardDocsPerQuery = new
ArrayList<>(queuePerQuery.resultIds(0).values());
+ shardDocsPerQuery.sort(Comparator.comparingInt(a ->
a.positionInResponse));
+ shardDocMap.put(queriesToCombineKeys[queryIndex], shardDocsPerQuery);
+ numFound = max(numFound, queryNumFound);
+ approximateTotalHits = max(approximateTotalHits,
queryApproximateTotalHits);
+ } // for each query to combine
+
+ rb.rsp.addToLog("hits", numFound);
+
+ SolrDocumentList responseDocs = new SolrDocumentList();
+ responseDocs.setNumFound(numFound);
+ responseDocs.setNumFoundExact(hitCountIsExact);
+ responseDocs.setStart(ss.getOffset());
+
+ rb.resultIds = computeResultIdsWithCombiner(rb, shardDocMap, responseDocs,
sortFields);
+ rb.setResponseDocs(responseDocs);
+
+ populateNextCursorMarkFromMergedShards(rb);
+
+ if (thereArePartialResults) {
+ rb.rsp
+ .getResponseHeader()
+ .asShallowMap()
+ .put(SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY,
Boolean.TRUE);
+ }
+ if (segmentTerminatedEarly != null) {
+ final Object existingSegmentTerminatedEarly =
+ rb.rsp
+ .getResponseHeader()
+
.get(SolrQueryResponse.RESPONSE_HEADER_SEGMENT_TERMINATED_EARLY_KEY);
+ if (existingSegmentTerminatedEarly == null) {
+ rb.rsp
+ .getResponseHeader()
+ .add(
+ SolrQueryResponse.RESPONSE_HEADER_SEGMENT_TERMINATED_EARLY_KEY,
+ segmentTerminatedEarly);
+ } else if (!Boolean.TRUE.equals(existingSegmentTerminatedEarly) &&
segmentTerminatedEarly) {
+ rb.rsp
+ .getResponseHeader()
+
.remove(SolrQueryResponse.RESPONSE_HEADER_SEGMENT_TERMINATED_EARLY_KEY);
+ rb.rsp
+ .getResponseHeader()
+
.add(SolrQueryResponse.RESPONSE_HEADER_SEGMENT_TERMINATED_EARLY_KEY, true);
+ }
+ }
+ if (maxHitsTerminatedEarly) {
+ rb.rsp
+ .getResponseHeader()
+
.add(SolrQueryResponse.RESPONSE_HEADER_MAX_HITS_TERMINATED_EARLY_KEY,
Boolean.TRUE);
+ if (approximateTotalHits > 0) {
+ rb.rsp
+ .getResponseHeader()
+ .add(
+ SolrQueryResponse.RESPONSE_HEADER_APPROXIMATE_TOTAL_HITS_KEY,
approximateTotalHits);
+ }
+ }
+ }
+
+ /**
+ * Populate shardInfo from mostly ShardResponse. Returns failedShardCount
(may be increased from
+ * the param).
+ */
+ private int addShardInfo(
+ NamedList<Object> shardInfo,
+ int failedShardCount,
+ ShardResponse srsp,
+ ResponseBuilder rb,
+ String queryKey,
+ SolrDocumentList docs) {
+ if (shardInfo == null) {
+ return failedShardCount;
+ }
+ SimpleOrderedMap<Object> nl = new SimpleOrderedMap<>();
+ NamedList<?> responseHeader;
+ if (srsp.getException() != null) {
+ Throwable t = srsp.getException();
+ if (t instanceof SolrServerException && t.getCause() != null) {
+ t = t.getCause();
+ }
+ nl.add("error", t.toString());
+ if (!rb.req.getCore().getCoreContainer().hideStackTrace()) {
+ StringWriter trace = new StringWriter();
+ t.printStackTrace(new PrintWriter(trace));
+ nl.add("trace", trace.toString());
+ }
+ if (!StrUtils.isNullOrEmpty(srsp.getShardAddress())) {
+ nl.add("shardAddress", srsp.getShardAddress());
+ }
+ } else {
+ responseHeader =
+ (NamedList<?>)
+ SolrResponseUtil.getSubsectionFromShardResponse(rb, srsp,
"responseHeader", false);
+ if (responseHeader == null) {
+ return failedShardCount;
+ }
+ final Object rhste =
+
responseHeader.get(SolrQueryResponse.RESPONSE_HEADER_SEGMENT_TERMINATED_EARLY_KEY);
+ if (rhste != null) {
+ nl.add(SolrQueryResponse.RESPONSE_HEADER_SEGMENT_TERMINATED_EARLY_KEY,
rhste);
+ }
+ final Object rhmhte =
+
responseHeader.get(SolrQueryResponse.RESPONSE_HEADER_MAX_HITS_TERMINATED_EARLY_KEY);
+ if (rhmhte != null) {
+
nl.add(SolrQueryResponse.RESPONSE_HEADER_MAX_HITS_TERMINATED_EARLY_KEY, rhmhte);
+ }
+ final Object rhath =
+
responseHeader.get(SolrQueryResponse.RESPONSE_HEADER_APPROXIMATE_TOTAL_HITS_KEY);
+ if (rhath != null) {
+ nl.add(SolrQueryResponse.RESPONSE_HEADER_APPROXIMATE_TOTAL_HITS_KEY,
rhath);
+ }
+ if (docs == null) {
+ return failedShardCount;
+ }
+ nl.add("numFound", docs.getNumFound());
+ nl.add("numFoundExact", docs.getNumFoundExact());
+ nl.add("maxScore", docs.getMaxScore());
+ nl.add("shardAddress", srsp.getShardAddress());
+ }
+
+ if (srsp.getSolrResponse() != null) {
+ nl.add("time", srsp.getSolrResponse().getElapsedTime());
+ }
+ nl.add("queryKey", queryKey);
+
+ // This ought to be better, but at least this ensures no duplicate keys in
JSON result
+ String shard = srsp.getShard() + "_" + queryKey;
+ shardInfo.add(shard, nl);
+ return failedShardCount;
+ }
+
+ /**
+ * Combines and sorts documents from multiple shards to create the final
result set. This method
+ * uses a combiner strategy to merge shard responses, then sorts the
resulting documents using a
+ * priority queue based on the request's sort specification. It handles
pagination (offset and
+ * count) and calculates the maximum score for the response.
+ *
+ * @param rb The ResponseBuilder containing the request and context, such as
sort specifications.
+ * @param shardDocMap A map from shard addresses to the list of documents
returned by each shard.
+ * @param responseDocs The final response document list, which will be
populated with null
+ * placeholders and have its max score set.
+ * @param sortFields An array of field for sorting to be applied.
+ * @return A map from document IDs to the corresponding ShardDoc objects for
the documents in the
+ * final sorted page of results.
+ */
+ protected Map<Object, ShardDoc> computeResultIdsWithCombiner(
+ ResponseBuilder rb,
+ Map<String, List<ShardDoc>> shardDocMap,
+ SolrDocumentList responseDocs,
+ SortField[] sortFields) {
+ String algorithm =
+ rb.req.getParams().get(CombinerParams.COMBINER_ALGORITHM,
CombinerParams.DEFAULT_COMBINER);
+ QueryAndResponseCombiner combinerStrategy =
+ QueryAndResponseCombiner.getImplementation(algorithm, combiners);
+ List<ShardDoc> combinedShardDocs = combinerStrategy.combine(shardDocMap,
rb.req.getParams());
+
+ // adding explanation for the ordered shard docs as debug info
+ if (rb.isDebugResults()) {
+ String[] queryKeys =
rb.req.getParams().getParams(CombinerParams.COMBINER_QUERY);
+ NamedList<Explanation> explanations =
+ combinerStrategy.getExplanations(
+ queryKeys, shardDocMap, combinedShardDocs, rb.req.getParams());
+ rb.addDebugInfo("combinerExplanations", explanations);
+ }
+ Map<String, ShardDoc> shardDocIdMap = new HashMap<>();
+ shardDocMap.forEach(
+ (shardKey, shardDocs) ->
+ shardDocs.forEach(shardDoc ->
shardDocIdMap.put(shardDoc.id.toString(), shardDoc)));
+
+ // creating a queue to sort basis on all the comparator and tie-break on
docId
+ Map<Object, ShardDoc> resultIds = new HashMap<>();
+ float maxScore = 0.0f;
+ final ShardFieldSortedHitQueue queue =
+ new ShardFieldSortedHitQueue(
+ sortFields,
+ rb.getSortSpec().getOffset() + rb.getSortSpec().getCount(),
+ rb.req.getSearcher()) {
+ @Override
+ protected boolean lessThan(ShardDoc docA, ShardDoc docB) {
+ int c = 0;
+ for (int i = 0; i < comparators.length && c == 0; i++) {
+ c =
+ (fields[i].getReverse())
+ ? comparators[i].compare(docB, docA)
+ : comparators[i].compare(docA, docB);
+ }
+
+ if (c == 0) {
+ c = docA.id.toString().compareTo(docB.id.toString());
+ }
+ return c < 0;
+ }
+ };
+ combinedShardDocs.forEach(queue::insertWithOverflow);
+
+ // get the resultSize as expected and fetch that shardDoc from the queue,
putting it in a map
+ int resultSize = max(0, queue.size() - rb.getSortSpec().getOffset());
+ for (int i = resultSize - 1; i >= 0; i--) {
+ ShardDoc shardDoc = queue.pop();
+ shardDoc.positionInResponse = i;
+ maxScore = max(maxScore, shardDoc.score);
+ if (Float.isNaN(shardDocIdMap.get(shardDoc.id.toString()).score)) {
+ shardDoc.score = Float.NaN;
+ }
+ resultIds.put(shardDoc.id.toString(), shardDoc);
+ }
+ responseDocs.setMaxScore(maxScore);
+ for (int i = 0; i < resultSize; i++) responseDocs.add(null);
+ return resultIds;
+ }
+}
diff --git
a/solr/core/src/java/org/apache/solr/handler/component/CombinedQueryResponseBuilder.java
b/solr/core/src/java/org/apache/solr/handler/component/CombinedQueryResponseBuilder.java
new file mode 100644
index 00000000000..ce6771b8e78
--- /dev/null
+++
b/solr/core/src/java/org/apache/solr/handler/component/CombinedQueryResponseBuilder.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.handler.component;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+
+/**
+ * The CombinedQueryResponseBuilder class extends the ResponseBuilder class
and is responsible for
+ * building a combined response for multiple SearchComponent objects. It
orchestrates the process of
+ * constructing the SolrQueryResponse by aggregating results from various
components.
+ */
+class CombinedQueryResponseBuilder extends ResponseBuilder {
+
+ final List<ResponseBuilder> responseBuilders = new ArrayList<>();
+
+ CombinedQueryResponseBuilder(
+ SolrQueryRequest req, SolrQueryResponse rsp, List<SearchComponent>
components) {
+ super(req, rsp, components);
+ }
+
+ /**
+ * Propagates all the properties from parent ResponseBuilder to the all the
children which are
+ * being set later after the CombinedQueryComponent is prepared.
+ */
+ final void propagate() {
+ responseBuilders.forEach(
+ thisRb -> {
+ thisRb.setNeedDocSet(isNeedDocSet());
+ thisRb.setNeedDocList(isNeedDocList());
+ thisRb.doFacets = doFacets;
+ thisRb.doHighlights = doHighlights;
+ thisRb.doExpand = doExpand;
+ thisRb.doTerms = doTerms;
+ thisRb.doStats = doStats;
+ thisRb.setDistribStatsDisabled(isDistribStatsDisabled());
+ });
+ }
+}
diff --git
a/solr/core/src/java/org/apache/solr/handler/component/CombinedQuerySearchHandler.java
b/solr/core/src/java/org/apache/solr/handler/component/CombinedQuerySearchHandler.java
new file mode 100644
index 00000000000..71029dd27d4
--- /dev/null
+++
b/solr/core/src/java/org/apache/solr/handler/component/CombinedQuerySearchHandler.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.handler.component;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.solr.common.params.CombinerParams;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+
+/**
+ * Extends the SearchHandler combining/fusing multiple queries (e.g. RRF) when
the {@link
+ * CombinerParams#COMBINER} param is provided. If it isn't, does nothing
special over SearchHandler.
+ *
+ * @see CombinedQueryComponent
+ */
+public class CombinedQuerySearchHandler extends SearchHandler {
+
+ /** Overrides to potentially return a custom {@link
CombinedQueryResponseBuilder}. */
+ @Override
+ protected ResponseBuilder newResponseBuilder(
+ SolrQueryRequest req, SolrQueryResponse rsp, List<SearchComponent>
components) {
+ if (req.getParams().getBool(CombinerParams.COMBINER, false)) {
+ var rb = new CombinedQueryResponseBuilder(req, rsp, components);
+ // CombinedQueryComponent is only designed to work with distributed
search.
+ rb.setForcedDistrib(true);
+ return rb;
+ }
+ return super.newResponseBuilder(req, rsp, components);
+ }
+
+ @Override
+ protected void postPrepareComponents(ResponseBuilder rb) {
+ super.postPrepareComponents(rb);
+ // propagate the CombinedQueryResponseBuilder's state to all subBuilders
after prepare
+ if (rb instanceof CombinedQueryResponseBuilder) {
+ var crb = (CombinedQueryResponseBuilder) rb;
+ crb.propagate();
+ }
+ }
+
+ /** Overrides the default list to include {@link CombinedQueryComponent}. */
+ @Override
+ protected List<String> getDefaultComponents() {
+ List<String> names = new ArrayList<>(super.getDefaultComponents());
+ String replaced = names.set(0, CombinedQueryComponent.COMPONENT_NAME);
+ assert replaced.equals(QueryComponent.COMPONENT_NAME);
+ return names;
+ }
+}
diff --git
a/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
b/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
index 8d30c87c203..6763b5a10c5 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
@@ -902,6 +902,67 @@ public class QueryComponent extends SearchComponent {
return true;
}
+ protected abstract static class ShardDocQueue {
+ public abstract boolean push(ShardDoc shardDoc);
+
+ public abstract Map<Object, ShardDoc> resultIds(int offset);
+ }
+ ;
+
+ protected ShardDocQueue newShardDocQueue(
+ SolrIndexSearcher searcher, SortField[] sortFields, Integer size) {
+ return new ShardDocQueue() {
+
+ // id to shard mapping, to eliminate any accidental dups
+ private final HashMap<Object, String> uniqueDoc = new HashMap<>();
+
+ private final ShardFieldSortedHitQueue queue =
+ new ShardFieldSortedHitQueue(sortFields, size, searcher);
+
+ @Override
+ public boolean push(ShardDoc shardDoc) {
+ final String prevShard = uniqueDoc.put(shardDoc.id, shardDoc.shard);
+ if (prevShard != null) {
+ // duplicate detected
+
+ // For now, just always use the first encountered since we can't
currently
+ // remove the previous one added to the priority queue. If we
switched
+ // to the Java5 PriorityQueue, this would be easier.
+ return false;
+ // make which duplicate is used deterministic based on shard
+ // if (prevShard.compareTo(shardDoc.shard) >= 0) {
+ // TODO: remove previous from priority queue
+ // return false;
+ // }
+ }
+
+ queue.insertWithOverflow(shardDoc);
+ return true;
+ }
+
+ @Override
+ public Map<Object, ShardDoc> resultIds(int offset) {
+ final Map<Object, ShardDoc> resultIds = new HashMap<>();
+
+ // The queue now has 0 -> queuesize docs, where queuesize <= start +
rows
+ // So we want to pop the last documents off the queue to get
+ // the docs offset -> queuesize
+ int resultSize = queue.size() - offset;
+ resultSize = Math.max(0, resultSize); // there may not be any docs in
range
+
+ for (int i = resultSize - 1; i >= 0; i--) {
+ ShardDoc shardDoc = queue.pop();
+ shardDoc.positionInResponse = i;
+ // Need the toString() for correlation with other lists that must
+ // be strings (like keys in highlighting, explain, etc)
+ resultIds.put(shardDoc.id.toString(), shardDoc);
+ }
+
+ return resultIds;
+ }
+ };
+ }
+
protected void mergeIds(ResponseBuilder rb, ShardRequest sreq) {
List<MergeStrategy> mergeStrategies = rb.getMergeStrategies();
if (mergeStrategies != null) {
@@ -944,14 +1005,10 @@ public class QueryComponent extends SearchComponent {
IndexSchema schema = rb.req.getSchema();
SchemaField uniqueKeyField = schema.getUniqueKeyField();
- // id to shard mapping, to eliminate any accidental dups
- HashMap<Object, String> uniqueDoc = new HashMap<>();
-
// Merge the docs via a priority queue so we don't have to sort *all* of
the
// documents... we only need to order the top (rows+start)
- final ShardFieldSortedHitQueue queue =
- new ShardFieldSortedHitQueue(
- sortFields, ss.getOffset() + ss.getCount(), rb.req.getSearcher());
+ final ShardDocQueue shardDocQueue =
+ newShardDocQueue(rb.req.getSearcher(), sortFields, ss.getOffset() +
ss.getCount());
NamedList<Object> shardInfo = null;
if (rb.req.getParams().getBool(ShardParams.SHARDS_INFO, false)) {
@@ -1122,23 +1179,6 @@ public class QueryComponent extends SearchComponent {
for (int i = 0; i < docs.size(); i++) {
SolrDocument doc = docs.get(i);
Object id = doc.getFieldValue(uniqueKeyField.getName());
-
- String prevShard = uniqueDoc.put(id, srsp.getShard());
- if (prevShard != null) {
- // duplicate detected
- numFound--;
-
- // For now, just always use the first encountered since we can't
currently
- // remove the previous one added to the priority queue. If we
switched
- // to the Java5 PriorityQueue, this would be easier.
- continue;
- // make which duplicate is used deterministic based on shard
- // if (prevShard.compareTo(srsp.shard) >= 0) {
- // TODO: remove previous from priority queue
- // continue;
- // }
- }
-
ShardDoc shardDoc = new ShardDoc();
shardDoc.id = id;
shardDoc.shard = srsp.getShard();
@@ -1157,42 +1197,18 @@ public class QueryComponent extends SearchComponent {
shardDoc.sortFieldValues = unmarshalledSortFieldValues;
- queue.insertWithOverflow(shardDoc);
+ if (!shardDocQueue.push(shardDoc)) {
+ numFound--;
+ }
} // end for-each-doc-in-response
} // end for-each-response
- // The queue now has 0 -> queuesize docs, where queuesize <= start + rows
- // So we want to pop the last documents off the queue to get
- // the docs offset -> queuesize
- int resultSize = queue.size() - ss.getOffset();
- resultSize = Math.max(0, resultSize); // there may not be any docs in range
-
- Map<Object, ShardDoc> resultIds = new HashMap<>();
- for (int i = resultSize - 1; i >= 0; i--) {
- ShardDoc shardDoc = queue.pop();
- shardDoc.positionInResponse = i;
- // Need the toString() for correlation with other lists that must
- // be strings (like keys in highlighting, explain, etc)
- resultIds.put(shardDoc.id.toString(), shardDoc);
- }
-
// Add hits for distributed requests
// https://issues.apache.org/jira/browse/SOLR-3518
rb.rsp.addToLog("hits", numFound);
- SolrDocumentList responseDocs = new SolrDocumentList();
- if (maxScore != null) responseDocs.setMaxScore(maxScore);
- responseDocs.setNumFound(numFound);
- responseDocs.setNumFoundExact(hitCountIsExact);
- responseDocs.setStart(ss.getOffset());
- // size appropriately
- for (int i = 0; i < resultSize; i++) responseDocs.add(null);
-
- // save these results in a private area so we can access them
- // again when retrieving stored fields.
- // TODO: use ResponseBuilder (w/ comments) or the request context?
- rb.resultIds = resultIds;
- rb.setResponseDocs(responseDocs);
+ setResultIdsAndResponseDocs(
+ rb, shardDocQueue, maxScore, numFound, hitCountIsExact,
ss.getOffset());
populateNextCursorMarkFromMergedShards(rb);
@@ -1238,6 +1254,30 @@ public class QueryComponent extends SearchComponent {
}
}
+ protected void setResultIdsAndResponseDocs(
+ ResponseBuilder rb,
+ ShardDocQueue shardDocQueue,
+ Float maxScore,
+ long numFound,
+ boolean hitCountIsExact,
+ int offset) {
+ final Map<Object, ShardDoc> resultIds = shardDocQueue.resultIds(offset);
+
+ final SolrDocumentList responseDocs = new SolrDocumentList();
+ if (maxScore != null) responseDocs.setMaxScore(maxScore);
+ responseDocs.setNumFound(numFound);
+ responseDocs.setNumFoundExact(hitCountIsExact);
+ responseDocs.setStart(offset);
+ // size appropriately
+ for (int i = 0; i < resultIds.size(); i++) responseDocs.add(null);
+
+ // save these results in a private area so we can access them
+ // again when retrieving stored fields.
+ // TODO: use ResponseBuilder (w/ comments) or the request context?
+ rb.resultIds = resultIds;
+ rb.setResponseDocs(responseDocs);
+ }
+
/**
* Inspects the state of the {@link ResponseBuilder} and populates the next
{@link
* ResponseBuilder#setNextCursorMark} as appropriate based on the merged
sort values from
diff --git
a/solr/core/src/java/org/apache/solr/handler/component/combine/QueryAndResponseCombiner.java
b/solr/core/src/java/org/apache/solr/handler/component/combine/QueryAndResponseCombiner.java
new file mode 100644
index 00000000000..c7d357fb12b
--- /dev/null
+++
b/solr/core/src/java/org/apache/solr/handler/component/combine/QueryAndResponseCombiner.java
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.handler.component.combine;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.TotalHits;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.handler.component.ShardDoc;
+import org.apache.solr.search.DocIterator;
+import org.apache.solr.search.DocSet;
+import org.apache.solr.search.DocSlice;
+import org.apache.solr.search.QueryResult;
+import org.apache.solr.util.plugin.NamedListInitializedPlugin;
+
+/**
+ * The QueryAndResponseCombiner class is an abstract base class for combining
query results and
+ * shard documents. It provides a framework for different algorithms to be
implemented for merging
+ * ranked lists and shard documents.
+ */
+public abstract class QueryAndResponseCombiner implements
NamedListInitializedPlugin {
+ /**
+ * Combines shard documents corresponding to multiple queries based on the
provided map.
+ *
+ * @param queriesDocMap a map where keys represent combiner query keys and
values are lists of
+ * ShardDocs for corresponding to each key
+ * @param solrParams params to be used when provided at query time
+ * @return a combined list of ShardDocs from all queries
+ */
+ public abstract List<ShardDoc> combine(
+ Map<String, List<ShardDoc>> queriesDocMap, SolrParams solrParams);
+
+ /**
+ * Simple combine query result list as a union.
+ *
+ * @param queryResults the query results to be combined
+ * @return the combined query result
+ */
+ public static QueryResult simpleCombine(List<QueryResult> queryResults) {
+ QueryResult combinedQueryResults = new QueryResult();
+ DocSet combinedDocSet = null;
+ Map<Integer, Float> uniqueDocIds = new HashMap<>();
+ long totalMatches = 0;
+ for (QueryResult queryResult : queryResults) {
+ DocIterator docs = queryResult.getDocList().iterator();
+ totalMatches = Math.max(totalMatches,
queryResult.getDocList().matches());
+ while (docs.hasNext()) {
+ uniqueDocIds.put(docs.nextDoc(), queryResult.getDocList().hasScores()
? docs.score() : 0f);
+ }
+ if (combinedDocSet == null) {
+ combinedDocSet = queryResult.getDocSet();
+ } else if (queryResult.getDocSet() != null) {
+ combinedDocSet = combinedDocSet.union(queryResult.getDocSet());
+ }
+ }
+ int combinedResultsLength = uniqueDocIds.size();
+ int[] combinedResultsDocIds = new int[combinedResultsLength];
+ float[] combinedResultScores = new float[combinedResultsLength];
+
+ int i = 0;
+ for (Map.Entry<Integer, Float> scoredDoc : uniqueDocIds.entrySet()) {
+ combinedResultsDocIds[i] = scoredDoc.getKey();
+ combinedResultScores[i] = scoredDoc.getValue();
+ i++;
+ }
+ DocSlice combinedResultSlice =
+ new DocSlice(
+ 0,
+ combinedResultsLength,
+ combinedResultsDocIds,
+ combinedResultScores,
+ Math.max(combinedResultsLength, totalMatches),
+ combinedResultScores.length > 0 ? combinedResultScores[0] : 0,
+ TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO);
+ combinedQueryResults.setDocList(combinedResultSlice);
+ combinedQueryResults.setDocSet(combinedDocSet);
+ return combinedQueryResults;
+ }
+
+ /**
+ * Retrieves a list of explanations for the given queries and results.
+ *
+ * @param queryKeys the keys associated with the queries
+ * @param queriesDocMap a map where keys represent combiner query keys and
values are lists of
+ * ShardDocs for corresponding to each key
+ * @param combinedQueriesDocs a list of ShardDocs after combiner operation
+ * @param solrParams params to be used when provided at query time
+ * @return a SimpleOrderedMap of explanations for the given queries and
results
+ */
+ public abstract SimpleOrderedMap<Explanation> getExplanations(
+ String[] queryKeys,
+ Map<String, List<ShardDoc>> queriesDocMap,
+ List<ShardDoc> combinedQueriesDocs,
+ SolrParams solrParams);
+
+ /**
+ * Retrieves an implementation of the QueryAndResponseCombiner based on the
specified algorithm.
+ *
+ * @param algorithm the combiner algorithm
+ * @param combiners The already initialised map of QueryAndResponseCombiner
+ * @return an instance of QueryAndResponseCombiner corresponding to the
specified algorithm.
+ * @throws SolrException if an unknown combiner algorithm is specified.
+ */
+ public static QueryAndResponseCombiner getImplementation(
+ String algorithm, Map<String, QueryAndResponseCombiner> combiners) {
+ if (combiners.get(algorithm) != null) {
+ return combiners.get(algorithm);
+ }
+ throw new SolrException(
+ SolrException.ErrorCode.BAD_REQUEST, "Unknown Combining algorithm: " +
algorithm);
+ }
+}
diff --git
a/solr/core/src/java/org/apache/solr/handler/component/combine/ReciprocalRankFusion.java
b/solr/core/src/java/org/apache/solr/handler/component/combine/ReciprocalRankFusion.java
new file mode 100644
index 00000000000..911c009c0d4
--- /dev/null
+++
b/solr/core/src/java/org/apache/solr/handler/component/combine/ReciprocalRankFusion.java
@@ -0,0 +1,167 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.handler.component.combine;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.StringJoiner;
+import java.util.stream.Collectors;
+import org.apache.lucene.search.Explanation;
+import org.apache.solr.common.params.CombinerParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.handler.component.ShardDoc;
+
+/**
+ * This class implements a query and response combiner that uses the
Reciprocal Rank Fusion (RRF)
+ * algorithm to combine multiple ranked lists into a single ranked list.
+ */
+public class ReciprocalRankFusion extends QueryAndResponseCombiner {
+
+ private int k;
+
+ public ReciprocalRankFusion() {
+ this.k = CombinerParams.DEFAULT_COMBINER_RRF_K;
+ }
+
+ @Override
+ public void init(NamedList<?> args) {
+ Object kParam = args.get("k");
+ if (kParam != null) {
+ this.k = Integer.parseInt(kParam.toString());
+ }
+ }
+
+ public int getK() {
+ return k;
+ }
+
+ /**
+ * Merges per-query ranked results using Reciprocal Rank Fusion (RRF).
+ *
+ * <p>Each query doc list is assumed to be ordered by descending relevance.
For a document at rank
+ * r in the list, the contribution is {@code 1 / (k + r)} where {@code k} is
read from {@link
+ * CombinerParams#COMBINER_RRF_K} or falls back to {@code this.k}.
Contributions for the same
+ * document ID across multiple queries (if found) are summed, and documents
are returned sorted by
+ * the fused score (descending).
+ *
+ * @param queriesDocMap per-query ranked results;
+ * @param solrParams parameters; optional {@link
CombinerParams#COMBINER_RRF_K} overrides k.
+ * @return one {@link ShardDoc} per unique document ID, ordered by fused
score.
+ */
+ @Override
+ public List<ShardDoc> combine(Map<String, List<ShardDoc>> queriesDocMap,
SolrParams solrParams) {
+ int kVal = solrParams.getInt(CombinerParams.COMBINER_RRF_K, this.k);
+ HashMap<String, Float> docIdToScore = new HashMap<>();
+ Map<String, ShardDoc> docIdToShardDoc = new HashMap<>();
+ List<ShardDoc> finalShardDocList = new ArrayList<>();
+ for (Map.Entry<String, List<ShardDoc>> shardDocEntry :
queriesDocMap.entrySet()) {
+ List<ShardDoc> shardDocList = shardDocEntry.getValue();
+ int ranking = 1;
+ while (ranking <= shardDocList.size()) {
+ String docId = shardDocList.get(ranking - 1).id.toString();
+ docIdToShardDoc.put(docId, shardDocList.get(ranking - 1));
+ float rrfScore = 1f / (kVal + ranking);
+ docIdToScore.compute(docId, (id, score) -> (score == null) ? rrfScore
: score + rrfScore);
+ ranking++;
+ }
+ }
+ List<Map.Entry<String, Float>> sortedByScoreDescending =
+ docIdToScore.entrySet().stream()
+ .sorted(
+ Comparator.comparing(Map.Entry<String, Float>::getValue,
Comparator.reverseOrder())
+ .thenComparing(Map.Entry::getKey))
+ .collect(Collectors.toList());
+ for (Map.Entry<String, Float> scoredDoc : sortedByScoreDescending) {
+ String docId = scoredDoc.getKey();
+ Float score = scoredDoc.getValue();
+ ShardDoc shardDoc = docIdToShardDoc.get(docId);
+ shardDoc.score = score;
+ finalShardDocList.add(shardDoc);
+ }
+ return finalShardDocList;
+ }
+
+ private Map<String, Integer[]> getRanks(
+ Collection<List<ShardDoc>> shardDocs, List<ShardDoc> combinedShardDocs) {
+ Map<String, Integer[]> docIdToRanks;
+ docIdToRanks = new HashMap<>();
+ for (ShardDoc shardDoc : combinedShardDocs) {
+ docIdToRanks.put(shardDoc.id.toString(), new Integer[shardDocs.size()]);
+ }
+ int docIdx = 0;
+ for (List<ShardDoc> shardDocList : shardDocs) {
+ int rank = 1;
+ for (ShardDoc shardDoc : shardDocList) {
+ String docId = shardDoc.id.toString();
+ docIdToRanks.get(docId)[docIdx] = rank;
+ rank++;
+ }
+ docIdx++;
+ }
+ return docIdToRanks;
+ }
+
+ @Override
+ public SimpleOrderedMap<Explanation> getExplanations(
+ String[] queryKeys,
+ Map<String, List<ShardDoc>> queriesDocMap,
+ List<ShardDoc> combinedQueriesDocs,
+ SolrParams solrParams) {
+ int kVal = solrParams.getInt(CombinerParams.COMBINER_RRF_K, this.k);
+ SimpleOrderedMap<Explanation> docIdsExplanations = new
SimpleOrderedMap<>();
+ Map<String, Integer[]> docIdToRanks = getRanks(queriesDocMap.values(),
combinedQueriesDocs);
+ for (ShardDoc shardDoc : combinedQueriesDocs) {
+ String docId = shardDoc.id.toString();
+ Integer[] rankPerQuery = docIdToRanks.get(docId);
+ Explanation fullDocIdExplanation =
+ Explanation.match(
+ shardDoc.score, getReciprocalRankFusionExplain(queryKeys,
rankPerQuery, kVal));
+ docIdsExplanations.add(docId, fullDocIdExplanation);
+ }
+ return docIdsExplanations;
+ }
+
+ private String getReciprocalRankFusionExplain(
+ String[] queryKeys, Integer[] rankPerQuery, int kVal) {
+ StringBuilder reciprocalRankFusionExplain = new StringBuilder();
+ StringJoiner scoreComponents = new StringJoiner(" + ");
+ for (Integer rank : rankPerQuery) {
+ if (rank != null) {
+ scoreComponents.add("1/(" + kVal + "+" + rank + ")");
+ }
+ }
+ reciprocalRankFusionExplain.append(scoreComponents);
+ reciprocalRankFusionExplain.append(" because its ranks were: ");
+ StringJoiner rankComponents = new StringJoiner(", ");
+ for (int i = 0; i < queryKeys.length; i++) {
+ Integer rank = rankPerQuery[i];
+ if (rank == null) {
+ rankComponents.add("not in the results for query(" + queryKeys[i] +
")");
+ } else {
+ rankComponents.add(rank + " for query(" + queryKeys[i] + ")");
+ }
+ }
+ reciprocalRankFusionExplain.append(rankComponents);
+ return reciprocalRankFusionExplain.toString();
+ }
+}
diff --git
a/solr/core/src/java/org/apache/solr/handler/component/combine/package-info.java
b/solr/core/src/java/org/apache/solr/handler/component/combine/package-info.java
new file mode 100644
index 00000000000..1098a9fed7f
--- /dev/null
+++
b/solr/core/src/java/org/apache/solr/handler/component/combine/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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.
+ */
+
+/**
+ * This contains the classes to combine the scores from search index results.
Multiple
+ * implementation of algorithms can be added to support them.
+ */
+package org.apache.solr.handler.component.combine;
diff --git
a/solr/core/src/test-files/solr/collection1/conf/solrconfig-combined-query.xml
b/solr/core/src/test-files/solr/collection1/conf/solrconfig-combined-query.xml
new file mode 100644
index 00000000000..b96ff9cd687
--- /dev/null
+++
b/solr/core/src/test-files/solr/collection1/conf/solrconfig-combined-query.xml
@@ -0,0 +1,168 @@
+<?xml version="1.0" ?>
+
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF 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.
+-->
+
+<!-- This is a minimal config file for Combined Query Component use.
+ -->
+
+<config>
+
+ <!-- Used to specify an alternate directory to hold all index data.
+ It defaults to "index" if not present, and should probably
+ not be changed if replication is in use. -->
+ <dataDir>${solr.data.dir:}</dataDir>
+
+ <schemaFactory class="ClassicIndexSchemaFactory"/>
+
+ <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
+
+ <statsCache class="org.apache.solr.search.stats.ExactStatsCache"/>
+
+ <updateHandler class="${solr.updateHandler:solr.DirectUpdateHandler2}">
+
+ <autoCommit>
+ <maxTime>${solr.autoCommit.maxTime:-1}</maxTime>
+ </autoCommit>
+
+ <updateLog enable="${enable.update.log:true}">
+ <str name="dir">${solr.ulog.dir:}</str>
+ </updateLog>
+
+ <commitWithin>
+ <softCommit>${solr.commitwithin.softcommit:true}</softCommit>
+ </commitWithin>
+
+ </updateHandler>
+
+ <query>
+ <!-- Maximum number of clauses in a boolean query... can affect
+ range or wildcard queries that expand to big boolean
+ queries. An exception is thrown if exceeded.
+ -->
+ <maxBooleanClauses>${solr.max.booleanClauses:1024}</maxBooleanClauses>
+
+ <!-- Cache specification for Filters or DocSets - unordered set of *all*
documents
+ that match a particular query.
+ -->
+ <filterCache
+ size="512"
+ initialSize="512"
+ autowarmCount="2"
+ async="${solr.filterCache.async:false}"/>
+
+ <queryResultCache
+ size="512"
+ initialSize="512"
+ autowarmCount="2"/>
+
+ <documentCache
+ size="512"
+ initialSize="512"
+ autowarmCount="0"/>
+
+ <cache name="perSegFilter"
+ class="solr.CaffeineCache"
+ size="10"
+ initialSize="0"
+ autowarmCount="10" />
+
+ <queryResultWindowSize>10</queryResultWindowSize>
+
+ <slowQueryThresholdMillis>2000</slowQueryThresholdMillis>
+
+ </query>
+
+ <queryResponseWriter name="xml" default="true"
+ class="solr.XMLResponseWriter" />
+
+ <!-- requestHandler plugins
+ -->
+ <requestHandler name="/select" class="solr.SearchHandler">
+ </requestHandler>
+
+ <requestHandler name="/search" class="solr.CombinedQuerySearchHandler">
+ </requestHandler>
+
+ <searchComponent class="solr.CombinedQueryComponent" name="combined_query">
+ <int name="maxCombinerQueries">2</int>
+ <lst name="combiners">
+ <lst name="test">
+ <str
name="class">org.apache.solr.handler.component.combine.TestCombiner</str>
+ <int name="var1">30</int>
+ <str name="var2">test</str>
+ </lst>
+ </lst>
+ </searchComponent>
+
+ <requestHandler name="/forcedDistribTest"
class="org.apache.solr.handler.component.CombinedQuerySolrCloudTest$ForcedDistribSearchHandler">
+ </requestHandler>
+
+ <searchComponent class="solr.HighlightComponent" name="highlight">
+ <highlighting>
+ <!-- Configure the standard fragmenter -->
+ <fragmenter name="gap" class="org.apache.solr.highlight.GapFragmenter"
default="true">
+ <lst name="defaults">
+ <int name="hl.fragsize">100</int>
+ </lst>
+ </fragmenter>
+
+ <fragmenter name="regex" class="org.apache.solr.highlight.RegexFragmenter">
+ <lst name="defaults">
+ <int name="hl.fragsize">70</int>
+ </lst>
+ </fragmenter>
+
+ <!-- Configure the standard formatter -->
+ <formatter name="html" class="org.apache.solr.highlight.HtmlFormatter"
default="true">
+ <lst name="defaults">
+ <str name="hl.simple.pre"><![CDATA[<em>]]></str>
+ <str name="hl.simple.post"><![CDATA[</em>]]></str>
+ </lst>
+ </formatter>
+
+ <!-- Configure the standard fragListBuilder -->
+ <fragListBuilder name="simple"
class="org.apache.solr.highlight.SimpleFragListBuilder" default="true"/>
+
+ <!-- Configure the standard fragmentsBuilder -->
+ <fragmentsBuilder name="simple"
class="org.apache.solr.highlight.SimpleFragmentsBuilder" default="true"/>
+ <fragmentsBuilder name="scoreOrder"
class="org.apache.solr.highlight.ScoreOrderFragmentsBuilder"/>
+
+ <boundaryScanner name="simple" class="solr.highlight.SimpleBoundaryScanner"
default="true">
+ <lst name="defaults">
+ <str name="hl.bs.maxScan">10</str>
+ <str name="hl.bs.chars">.,!? 	 </str>
+ </lst>
+ </boundaryScanner>
+
+ <boundaryScanner name="breakIterator"
class="solr.highlight.BreakIteratorBoundaryScanner">
+ <lst name="defaults">
+ <str name="hl.bs.type">WORD</str>
+ <str name="hl.bs.language">en</str>
+ <str name="hl.bs.country">US</str>
+ </lst>
+ </boundaryScanner>
+ </highlighting>
+ </searchComponent>
+
+ <initParams path="/select,/search">
+ <lst name="defaults">
+ <str name="df">text</str>
+ </lst>
+ </initParams>
+
+</config>
diff --git
a/solr/core/src/test/org/apache/solr/handler/component/CombinedQueryComponentTest.java
b/solr/core/src/test/org/apache/solr/handler/component/CombinedQueryComponentTest.java
new file mode 100644
index 00000000000..224c59bb1b7
--- /dev/null
+++
b/solr/core/src/test/org/apache/solr/handler/component/CombinedQueryComponentTest.java
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.handler.component;
+
+import static org.apache.solr.common.params.CursorMarkParams.CURSOR_MARK_START;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.solr.BaseDistributedSearchTestCase;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * The CombinedQueryComponentTest class is an integration test suite for the
CombinedQueryComponent
+ * in Solr. It verifies the functionality of the component by performing few
basic queries in single
+ * sharded mode and validating the responses including limitations and
combiner plugin.
+ */
+public class CombinedQueryComponentTest extends BaseDistributedSearchTestCase {
+
+ private static final int NUM_DOCS = 10;
+ private static final String vectorField = "vector";
+
+ public CombinedQueryComponentTest() {
+ super();
+ fixShardCount(1);
+ }
+
+ /**
+ * Sets up the test class by initializing the core and setting system
properties. This method is
+ * executed before all test methods in the class.
+ */
+ @BeforeClass
+ public static void setUpClass() {
+ schemaString = "schema-vector-catchall.xml";
+ configString = "solrconfig-combined-query.xml";
+ System.setProperty("validateAfterInactivity", "200");
+ System.setProperty("solr.httpclient.retries", "0");
+ System.setProperty("distribUpdateSoTimeout", "5000");
+ }
+
+ /**
+ * Prepares Solr input documents for indexing, including adding sample data
and vector fields.
+ * This method populates the Solr index with test data, including text,
title, and vector fields.
+ * The vector fields are used to calculate cosine distance for testing
purposes.
+ *
+ * @throws Exception if any error occurs during the indexing process.
+ */
+ private synchronized void prepareIndexDocs() throws Exception {
+ List<SolrInputDocument> docs = new ArrayList<>();
+ for (int i = 1; i <= NUM_DOCS; i++) {
+ SolrInputDocument doc = new SolrInputDocument();
+ doc.addField("id", Integer.toString(i));
+ doc.addField("text", "test text for doc " + i);
+ doc.addField("title", "title test for doc " + i);
+ doc.addField("mod3_idv", (i % 3));
+ docs.add(doc);
+ }
+ // cosine distance vector1= 1.0
+ docs.get(0).addField(vectorField, Arrays.asList(1f, 2f, 3f, 4f));
+ // cosine distance vector1= 0.998
+ docs.get(1).addField(vectorField, Arrays.asList(1.5f, 2.5f, 3.5f, 4.5f));
+ // cosine distance vector1= 0.992
+ docs.get(2).addField(vectorField, Arrays.asList(7.5f, 15.5f, 17.5f,
22.5f));
+ // cosine distance vector1= 0.999
+ docs.get(3).addField(vectorField, Arrays.asList(1.4f, 2.4f, 3.4f, 4.4f));
+ // cosine distance vector1= 0.862
+ docs.get(4).addField(vectorField, Arrays.asList(30f, 22f, 35f, 20f));
+ // cosine distance vector1= 0.756
+ docs.get(5).addField(vectorField, Arrays.asList(40f, 1f, 1f, 200f));
+ // cosine distance vector1= 0.970
+ docs.get(6).addField(vectorField, Arrays.asList(5f, 10f, 20f, 40f));
+ // cosine distance vector1= 0.515
+ docs.get(7).addField(vectorField, Arrays.asList(120f, 60f, 30f, 15f));
+ // cosine distance vector1= 0.554
+ docs.get(8).addField(vectorField, Arrays.asList(200f, 50f, 100f, 25f));
+ // cosine distance vector1= 0.997
+ docs.get(9).addField(vectorField, Arrays.asList(1.8f, 2.5f, 3.7f, 4.9f));
+ del("*:*");
+ for (SolrInputDocument doc : docs) {
+ indexDoc(doc);
+ }
+ commit();
+ }
+
+ /** Performs a single lexical query using the provided JSON request and
verifies the response. */
+ public void testSingleLexicalQuery() throws Exception {
+ prepareIndexDocs();
+ QueryResponse rsp =
+ query(
+ CommonParams.JSON,
+ "{\"queries\":"
+ + "{\"lexical1\":{\"lucene\":{\"query\":\"title:title test for
doc 5\"}}},"
+ + "\"limit\":5,"
+ + "\"fields\":[\"id\",\"score\",\"title\"],"
+ +
"\"params\":{\"combiner\":true,\"combiner.query\":[\"lexical1\"]}}",
+ CommonParams.QT,
+ "/search");
+ assertEquals(5, rsp.getResults().size());
+ }
+
+ /** Performs multiple lexical queries and verifies the results. */
+ public void testMultipleLexicalQueryWithDebug() throws Exception {
+ prepareIndexDocs();
+ QueryResponse rsp =
+ query(
+ CommonParams.JSON,
+ "{\"queries\":"
+ + "{\"lexical1\":{\"lucene\":{\"query\":\"title:title test for
doc 1\"}},"
+ + "\"lexical2\":{\"lucene\":{\"query\":\"text:test text for
doc 2\"}}},"
+ + "\"limit\":5,"
+ + "\"fields\":[\"id\",\"score\",\"title\"],"
+ +
"\"params\":{\"combiner\":true,\"debug\":[\"results\"],\"combiner.query\":[\"lexical1\",\"lexical2\"],"
+ + "\"rid\": \"test-1\"}}",
+ CommonParams.QT,
+ "/search");
+ assertEquals(10, rsp.getResults().getNumFound());
+ assertTrue(rsp.getDebugMap().containsKey("combinerExplanations"));
+ }
+
+ /** Test no results in combined queries. */
+ @Test
+ public void testNoResults() throws Exception {
+ prepareIndexDocs();
+ QueryResponse rsp =
+ query(
+ CommonParams.JSON,
+ "{\"queries\":"
+ + "{\"lexical1\":{\"lucene\":{\"query\":\"title:Solr is the
blazing-fast, open source search platform\"}},"
+ + "\"lexical2\":{\"lucene\":{\"query\":\"text:Solr powers the
search\"}}},"
+ + "\"limit\":5,"
+ + "\"fields\":[\"id\",\"score\",\"title\"],"
+ +
"\"params\":{\"combiner\":true,\"combiner.query\":[\"lexical1\",\"lexical2\"]}}",
+ CommonParams.QT,
+ "/search");
+ assertEquals(0, rsp.getResults().size());
+ }
+
+ /** Test max combiner queries limit set from solrconfig to 2. */
+ @Test
+ public void testMaxQueriesLimit() throws Exception {
+ prepareIndexDocs();
+ RuntimeException exceptionThrown =
+ expectThrows(
+ SolrException.class,
+ () ->
+ query(
+ CommonParams.JSON,
+ "{\"queries\":"
+ + "{\"lexical1\":{\"lucene\":{\"query\":\"id:(2^=2 OR
3^=1)\"}},"
+ + "\"vector\":{\"knn\":{ \"f\": \"vector\", \"topK\":
5, \"query\": \"[1.0, 2.0, 3.0, 4.0]\"}},"
+ + "\"lexical2\":{\"lucene\":{\"query\":\"text:test
text for doc 2\"}}},"
+ + "\"limit\":5,"
+ + "\"fields\":[\"id\",\"score\",\"title\"],"
+ +
"\"params\":{\"combiner\":true,\"combiner.query\":[\"lexical1\",\"vector\",
\"lexical2\"]}}",
+ CommonParams.QT,
+ "/search"));
+ assertTrue(exceptionThrown.getMessage().contains("Too many queries to
combine: limit is 2"));
+ }
+
+ /**
+ * Test to ensure the TestCombiner Algorithm is injected through solrconfigs
and is being executed
+ * when sent the command through SolrParams
+ */
+ @Test
+ public void testCombinerPlugin() throws Exception {
+ prepareIndexDocs();
+ QueryResponse rsp =
+ query(
+ CommonParams.JSON,
+ "{\"queries\":"
+ + "{\"lexical1\":{\"lucene\":{\"query\":\"title:title test for
doc 1\"}},"
+ + "\"lexical2\":{\"lucene\":{\"query\":\"text:test text for
doc 2\"}}},"
+ + "\"limit\":5,"
+ + "\"fields\":[\"id\",\"score\",\"title\"],"
+ +
"\"params\":{\"combiner\":true,\"combiner.algorithm\":test,\"combiner.query\""
+ + ":[\"lexical1\",\"lexical2\"],\"debug\":[\"results\"]}}",
+ CommonParams.QT,
+ "/search");
+ assertEquals(10, rsp.getResults().getNumFound());
+ assertEquals(
+ "org.apache.lucene.search.Explanation:30 = this is test combiner\n",
+ ((SimpleOrderedMap<?>) rsp.getDebugMap().get("combinerExplanations"))
+ .get("combinerDetails"));
+ }
+
+ /**
+ * Tests that using unsupported features with Combined Queries throws the
expected exception.
+ *
+ * <p>This test case verifies that requests for Combined Queries that
include either the
+ * 'cursorMark' or 'group' parameters.
+ */
+ @Test
+ public void testNonEnabledFeature() throws Exception {
+ prepareIndexDocs();
+ String combinedQueryStr =
+ "{\"queries\":"
+ + "{\"lexical1\":{\"lucene\":{\"query\":\"title:title test for doc
1\"}},"
+ + "\"lexical2\":{\"lucene\":{\"query\":\"text:test text for doc
2\"}}},"
+ + "\"sort\":\"id asc\","
+ + "\"fields\":[\"id\",\"score\",\"title\"],"
+ +
"\"params\":{\"combiner\":true,\"combiner.algorithm\":test,\"combiner.query\":[\"lexical1\",\"lexical2\"]}}";
+
+ RuntimeException exceptionThrown =
+ expectThrows(
+ SolrException.class,
+ () ->
+ query(
+ CommonParams.JSON,
+ combinedQueryStr,
+ CommonParams.QT,
+ "/search",
+ "cursorMark",
+ CURSOR_MARK_START));
+ assertTrue(
+ exceptionThrown.getMessage().contains("Unsupported functionality for
Combined Queries."));
+ exceptionThrown =
+ expectThrows(
+ SolrException.class,
+ () ->
+ query(
+ CommonParams.JSON,
+ combinedQueryStr,
+ CommonParams.QT,
+ "/search",
+ "group",
+ "true"));
+ assertTrue(
+ exceptionThrown.getMessage().contains("Unsupported functionality for
Combined Queries."));
+ }
+}
diff --git
a/solr/core/src/test/org/apache/solr/handler/component/CombinedQuerySearchHandlerTest.java
b/solr/core/src/test/org/apache/solr/handler/component/CombinedQuerySearchHandlerTest.java
new file mode 100644
index 00000000000..97ba6ce9f5c
--- /dev/null
+++
b/solr/core/src/test/org/apache/solr/handler/component/CombinedQuerySearchHandlerTest.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.handler.component;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.params.CombinerParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/** The type Combined query search handler test. */
+public class CombinedQuerySearchHandlerTest extends SolrTestCaseJ4 {
+
+ /**
+ * Before tests.
+ *
+ * @throws Exception the exception
+ */
+ @BeforeClass
+ public static void beforeTests() throws Exception {
+ initCore("solrconfig.xml", "schema.xml");
+ }
+
+ /** Test combined component init in search components list. */
+ @Test
+ public void testCombinedComponentInit() {
+ SolrCore core = h.getCore();
+
+ try (CombinedQuerySearchHandler handler = new
CombinedQuerySearchHandler()) {
+ handler.init(new NamedList<>());
+ handler.inform(core);
+ assertEquals(9, handler.getComponents().size());
+ assertEquals(
+ core.getSearchComponent(CombinedQueryComponent.COMPONENT_NAME),
+ handler.getComponents().get(0));
+ } catch (IOException e) {
+ fail("Exception when closing CombinedQuerySearchHandler");
+ }
+ }
+
+ /** Test combined response buildr type create dynamically. */
+ @Test
+ public void testCombinedResponseBuilder() {
+ SolrQueryRequest request = req("q", "testQuery");
+ try (CombinedQuerySearchHandler handler = new
CombinedQuerySearchHandler()) {
+ assertFalse(
+ handler.newResponseBuilder(request, new SolrQueryResponse(), new
ArrayList<>())
+ instanceof CombinedQueryResponseBuilder);
+ request = req("q", "testQuery", CombinerParams.COMBINER, "true");
+ assertTrue(
+ handler.newResponseBuilder(request, new SolrQueryResponse(), new
ArrayList<>())
+ instanceof CombinedQueryResponseBuilder);
+ } catch (IOException e) {
+ fail("Exception when closing CombinedQuerySearchHandler");
+ }
+ }
+}
diff --git
a/solr/core/src/test/org/apache/solr/handler/component/CombinedQuerySolrCloudTest.java
b/solr/core/src/test/org/apache/solr/handler/component/CombinedQuerySolrCloudTest.java
new file mode 100644
index 00000000000..ea674dbc739
--- /dev/null
+++
b/solr/core/src/test/org/apache/solr/handler/component/CombinedQuerySolrCloudTest.java
@@ -0,0 +1,284 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.handler.component;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ShardParams;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.util.RTimerTree;
+import org.junit.Test;
+
+/**
+ * Contains integration tests for the combined query functionality in a
SolrCloud environment. This
+ * class extends AbstractFullDistribZkTestBase to leverage the distributed
testing framework. The
+ * tests cover various scenarios including single and multiple lexical
queries, sorting, pagination,
+ * faceting, and highlighting similar to {@link
DistributedCombinedQueryComponentTest}
+ */
+public class CombinedQuerySolrCloudTest extends AbstractFullDistribZkTestBase {
+
+ private static final int NUM_DOCS = 10;
+ private static final String vectorField = "vector";
+
+ public CombinedQuerySolrCloudTest() {
+ super();
+ sliceCount = 2;
+ fixShardCount(2);
+ }
+
+ @Override
+ protected String getCloudSchemaFile() {
+ return "schema-vector-catchall.xml";
+ }
+
+ @Override
+ protected String getCloudSolrConfig() {
+ return "solrconfig-combined-query.xml";
+ }
+
+ private synchronized void prepareIndexDocs() throws Exception {
+ List<SolrInputDocument> docs = new ArrayList<>();
+ for (int i = 1; i <= NUM_DOCS; i++) {
+ SolrInputDocument doc = new SolrInputDocument();
+ doc.addField("id", Integer.toString(i));
+ doc.addField("text", "test text for doc " + i);
+ doc.addField("title", "title test for doc " + i);
+ doc.addField("mod3_idv", (i % 3));
+ docs.add(doc);
+ }
+ // cosine distance vector1= 1.0
+ docs.get(0).addField(vectorField, Arrays.asList(1f, 2f, 3f, 4f));
+ // cosine distance vector1= 0.998
+ docs.get(1).addField(vectorField, Arrays.asList(1.5f, 2.5f, 3.5f, 4.5f));
+ // cosine distance vector1= 0.992
+ docs.get(2).addField(vectorField, Arrays.asList(7.5f, 15.5f, 17.5f,
22.5f));
+ // cosine distance vector1= 0.999
+ docs.get(3).addField(vectorField, Arrays.asList(1.4f, 2.4f, 3.4f, 4.4f));
+ // cosine distance vector1= 0.862
+ docs.get(4).addField(vectorField, Arrays.asList(30f, 22f, 35f, 20f));
+ // cosine distance vector1= 0.756
+ docs.get(5).addField(vectorField, Arrays.asList(40f, 1f, 1f, 200f));
+ // cosine distance vector1= 0.970
+ docs.get(6).addField(vectorField, Arrays.asList(5f, 10f, 20f, 40f));
+ // cosine distance vector1= 0.515
+ docs.get(7).addField(vectorField, Arrays.asList(120f, 60f, 30f, 15f));
+ // cosine distance vector1= 0.554
+ docs.get(8).addField(vectorField, Arrays.asList(200f, 50f, 100f, 25f));
+ // cosine distance vector1= 0.997
+ docs.get(9).addField(vectorField, Arrays.asList(1.8f, 2.5f, 3.7f, 4.9f));
+ del("*:*");
+ for (SolrInputDocument doc : docs) {
+ indexDoc(doc);
+ }
+ commit();
+ }
+
+ @Test
+ public void testSingleLexicalQuery() throws Exception {
+ prepareIndexDocs();
+ QueryResponse rsp =
+ query(
+ CommonParams.JSON,
+ "{\"queries\":"
+ + "{\"lexical1\":{\"lucene\":{\"query\":\"id:2^10\"}}},"
+ + "\"limit\":5,"
+ + "\"fields\":[\"id\",\"score\",\"title\"],"
+ +
"\"params\":{\"combiner\":true,\"combiner.query\":[\"lexical1\"]}}",
+ CommonParams.QT,
+ "/search");
+ assertEquals(1, rsp.getResults().size());
+ assertFieldValues(rsp.getResults(), id, "2");
+ }
+
+ /**
+ * Tests multiple lexical queries using the solr cloud client against
control client.
+ *
+ * @throws Exception if any error occurs during the test execution
+ */
+ @Test
+ public void testMultipleLexicalQuery() throws Exception {
+ prepareIndexDocs();
+ String jsonQuery =
+ "{\"queries\":"
+ + "{\"lexical1\":{\"lucene\":{\"query\":\"id:(2^2 OR 3^1 OR 6^2 OR
5^1)\"}},"
+ + "\"lexical2\":{\"lucene\":{\"query\":\"id:(8^1 OR 5^2 OR 7^3 OR
10^2)\"}}},"
+ + "\"limit\":5,"
+ + "\"fields\":[\"id\",\"score\",\"title\"],"
+ +
"\"params\":{\"combiner\":true,\"combiner.query\":[\"lexical1\",\"lexical2\"]}}";
+ QueryResponse rsp = query(CommonParams.JSON, jsonQuery, CommonParams.QT,
"/search");
+ assertEquals(5, rsp.getResults().size());
+ assertFieldValues(rsp.getResults(), id, "5", "7", "2", "6", "3");
+ }
+
+ /**
+ * Test multiple query execution with sort.
+ *
+ * @throws Exception the exception
+ */
+ @Test
+ public void testMultipleQueryWithSort() throws Exception {
+ prepareIndexDocs();
+ String jsonQuery =
+ "{\"queries\":"
+ + "{\"lexical1\":{\"lucene\":{\"query\":\"id:(2^2 OR 3^1 OR 6^2 OR
5^1)\"}},"
+ + "\"lexical2\":{\"lucene\":{\"query\":\"id:(8^1 OR 5^2 OR 7^3 OR
10^1)\"}}},"
+ + "\"limit\":5,\"sort\":\"mod3_idv desc, score desc\""
+ + "\"fields\":[\"id\",\"score\",\"title\"],"
+ +
"\"params\":{\"combiner\":true,\"combiner.query\":[\"lexical1\",\"lexical2\"]}}";
+ QueryResponse rsp = query(CommonParams.JSON, jsonQuery, CommonParams.QT,
"/search");
+ assertEquals(5, rsp.getResults().size());
+ assertFieldValues(rsp.getResults(), id, "5", "2", "8", "7", "10");
+ }
+
+ /**
+ * Tests the hybrid query functionality of the system with various setting
of pagination.
+ *
+ * @throws Exception if any unexpected error occurs during the test
execution.
+ */
+ @Test
+ public void testHybridQueryWithPagination() throws Exception {
+ prepareIndexDocs();
+ QueryResponse rsp =
+ query(
+ CommonParams.JSON,
+ "{\"queries\":"
+ + "{\"lexical1\":{\"lucene\":{\"query\":\"id:(2^2 OR 3^1 OR
6^2 OR 5^1)\"}},"
+ + "\"lexical2\":{\"lucene\":{\"query\":\"id:(8^1 OR 5^2 OR 7^3
OR 10^2)\"}}},"
+ + "\"fields\":[\"id\",\"score\",\"title\"],"
+ +
"\"params\":{\"combiner\":true,\"combiner.query\":[\"lexical1\",\"lexical2\"]}}",
+ CommonParams.QT,
+ "/search");
+ assertFieldValues(rsp.getResults(), id, "5", "7", "2", "6", "3", "10",
"8");
+ rsp =
+ query(
+ CommonParams.JSON,
+ "{\"queries\":"
+ + "{\"lexical1\":{\"lucene\":{\"query\":\"id:(2^2 OR 3^1 OR
6^2 OR 5^1)\"}},"
+ + "\"lexical2\":{\"lucene\":{\"query\":\"id:(8^1 OR 5^2 OR 7^3
OR 10^2)\"}}},"
+ + "\"limit\":4,"
+ + "\"fields\":[\"id\",\"score\",\"title\"],"
+ +
"\"params\":{\"combiner\":true,\"combiner.query\":[\"lexical1\",\"lexical2\"]}}",
+ CommonParams.QT,
+ "/search");
+ assertFieldValues(rsp.getResults(), id, "5", "7", "2", "6");
+ rsp =
+ query(
+ CommonParams.JSON,
+ "{\"queries\":"
+ + "{\"lexical1\":{\"lucene\":{\"query\":\"id:(2^2 OR 3^1 OR
6^2 OR 5^1)\"}},"
+ + "\"lexical2\":{\"lucene\":{\"query\":\"id:(8^1 OR 5^2 OR 7^3
OR 10^2)\"}}},"
+ + "\"limit\":4,\"offset\":3,"
+ + "\"fields\":[\"id\",\"score\",\"title\"],"
+ +
"\"params\":{\"combiner\":true,\"combiner.query\":[\"lexical1\",\"lexical2\"]}}",
+ CommonParams.QT,
+ "/search");
+ assertEquals(4, rsp.getResults().size());
+ assertFieldValues(rsp.getResults(), id, "6", "3", "10", "8");
+ }
+
+ /**
+ * Tests the single query functionality with faceting only.
+ *
+ * @throws Exception if any unexpected error occurs during the test
execution.
+ */
+ @Test
+ public void testQueryWithFaceting() throws Exception {
+ prepareIndexDocs();
+ String jsonQuery =
+ "{\"queries\":"
+ + "{\"lexical\":{\"lucene\":{\"query\":\"id:(2^2 OR 3^1 OR 6^2 OR
5^1)\"}}},"
+ + "\"limit\":3,\"offset\":1"
+ + "\"fields\":[\"id\",\"score\",\"title\"],"
+ +
"\"params\":{\"combiner\":true,\"facet\":true,\"facet.field\":\"mod3_idv\",\"facet.mincount\":1,"
+ + "\"combiner.query\":[\"lexical\"]}}";
+ QueryResponse rsp = query(CommonParams.JSON, jsonQuery, CommonParams.QT,
"/search");
+ assertEquals(3, rsp.getResults().size());
+ assertEquals(4, rsp.getResults().getNumFound());
+ assertEquals("[0 (2), 2 (2)]",
rsp.getFacetFields().get(0).getValues().toString());
+ }
+
+ /**
+ * Tests the combined query feature with faceting and highlighting.
+ *
+ * @throws Exception if any unexpected error occurs during the test
execution.
+ */
+ @Test
+ public void testQueriesWithFacetAndHighlights() throws Exception {
+ prepareIndexDocs();
+ String jsonQuery =
+ "{\"queries\":"
+ + "{\"lexical1\":{\"lucene\":{\"query\":\"id:(2^2 OR 3^1 OR 6^2 OR
5^1)\"}},"
+ + "\"lexical2\":{\"lucene\":{\"query\":\"id:(8^1 OR 5^2 OR 7^3 OR
10^2)\"}}},"
+ + "\"limit\":4,"
+ + "\"fields\":[\"id\",\"score\",\"title\"],"
+ +
"\"params\":{\"combiner\":true,\"facet\":true,\"facet.field\":\"mod3_idv\","
+ + "\"combiner.query\":[\"lexical1\",\"lexical2\"], \"hl\": true,"
+ + "\"hl.fl\": \"title\",\"hl.q\":\"test doc\"}}";
+ QueryResponse rsp = query(CommonParams.JSON, jsonQuery, CommonParams.QT,
"/search");
+ assertEquals(4, rsp.getResults().size());
+ assertFieldValues(rsp.getResults(), id, "5", "7", "2", "6");
+ assertEquals("mod3_idv", rsp.getFacetFields().get(0).getName());
+ assertEquals("[2 (3), 0 (2), 1 (2)]",
rsp.getFacetFields().get(0).getValues().toString());
+ assertEquals(4, rsp.getHighlighting().size());
+ assertEquals(
+ "title <em>test</em> for <em>doc</em> 2",
+ rsp.getHighlighting().get("2").get("title").get(0));
+ assertEquals(
+ "title <em>test</em> for <em>doc</em> 5",
+ rsp.getHighlighting().get("5").get("title").get(0));
+ }
+
+ /** To test that we can force distrib */
+ public static class ForcedDistribSearchHandler extends SearchHandler {
+ @Override
+ protected ResponseBuilder newResponseBuilder(
+ SolrQueryRequest req, SolrQueryResponse rsp, List<SearchComponent>
components) {
+ ResponseBuilder rb = super.newResponseBuilder(req, rsp, components);
+ rb.setForcedDistrib(true);
+ return rb;
+ }
+
+ @Override
+ protected void processComponentsLocal(
+ SolrQueryRequest req,
+ SolrQueryResponse rsp,
+ ResponseBuilder rb,
+ RTimerTree timer,
+ List<SearchComponent> components)
+ throws IOException {
+ if (req.getParams().getPrimitiveBool(ShardParams.IS_SHARD)) {
+ super.processComponentsLocal(req, rsp, rb, timer, components);
+ } else {
+ fail("forcing distrib didn't work");
+ }
+ }
+ }
+
+ /** Tests {@link ResponseBuilder#setForcedDistrib(boolean)} had the desired
effect. */
+ @Test
+ public void testForcedDistrib() throws Exception {
+ QueryResponse rsp = query("qt", "/forcedDistribTest", "q", "*:*", "rows",
"0");
+ // ForcedDistribSearchHandler would trigger a failure if this didn't work
+ }
+}
diff --git
a/solr/core/src/test/org/apache/solr/handler/component/DistributedCombinedQueryComponentTest.java
b/solr/core/src/test/org/apache/solr/handler/component/DistributedCombinedQueryComponentTest.java
new file mode 100644
index 00000000000..8a1656b4976
--- /dev/null
+++
b/solr/core/src/test/org/apache/solr/handler/component/DistributedCombinedQueryComponentTest.java
@@ -0,0 +1,293 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.handler.component;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.solr.BaseDistributedSearchTestCase;
+import org.apache.solr.client.solrj.impl.Http2SolrClient;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.params.CommonParams;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * The DistributedCombinedQueryComponentTest class is a JUnit test suite that
evaluates the
+ * functionality of the CombinedQueryComponent in a Solr distributed search
environment. It focuses
+ * on testing the integration of combiner queries with different
configurations.
+ */
+public class DistributedCombinedQueryComponentTest extends
BaseDistributedSearchTestCase {
+
+ private static final int NUM_DOCS = 10;
+ private static final String vectorField = "vector";
+
+ /**
+ * Sets up the test class by initializing the core and setting system
properties. This method is
+ * executed before all test methods in the class.
+ */
+ @BeforeClass
+ public static void setUpClass() {
+ schemaString = "schema-vector-catchall.xml";
+ configString = "solrconfig-combined-query.xml";
+ System.setProperty("validateAfterInactivity", "200");
+ System.setProperty("solr.httpclient.retries", "0");
+ System.setProperty("distribUpdateSoTimeout", "5000");
+ }
+
+ /**
+ * Prepares Solr input documents for indexing, including adding sample data
and vector fields.
+ * This method populates the Solr index with test data, including text,
title, and vector fields.
+ * The vector fields are used to calculate cosine distance for testing
purposes.
+ *
+ * @throws Exception if any error occurs during the indexing process.
+ */
+ private synchronized void prepareIndexDocs() throws Exception {
+ List<SolrInputDocument> docs = new ArrayList<>();
+ fixShardCount(2);
+ for (int i = 1; i <= NUM_DOCS; i++) {
+ SolrInputDocument doc = new SolrInputDocument();
+ doc.addField("id", Integer.toString(i));
+ doc.addField("text", "test text for doc " + i);
+ doc.addField("title", "title test for doc " + i);
+ doc.addField("mod3_idv", (i % 3));
+ docs.add(doc);
+ }
+ // cosine distance vector1= 1.0
+ docs.get(0).addField(vectorField, Arrays.asList(1f, 2f, 3f, 4f));
+ // cosine distance vector1= 0.998
+ docs.get(1).addField(vectorField, Arrays.asList(1.5f, 2.5f, 3.5f, 4.5f));
+ // cosine distance vector1= 0.992
+ docs.get(2).addField(vectorField, Arrays.asList(7.5f, 15.5f, 17.5f,
22.5f));
+ // cosine distance vector1= 0.999
+ docs.get(3).addField(vectorField, Arrays.asList(1.4f, 2.4f, 3.4f, 4.4f));
+ // cosine distance vector1= 0.862
+ docs.get(4).addField(vectorField, Arrays.asList(30f, 22f, 35f, 20f));
+ // cosine distance vector1= 0.756
+ docs.get(5).addField(vectorField, Arrays.asList(40f, 1f, 1f, 200f));
+ // cosine distance vector1= 0.970
+ docs.get(6).addField(vectorField, Arrays.asList(5f, 10f, 20f, 40f));
+ // cosine distance vector1= 0.515
+ docs.get(7).addField(vectorField, Arrays.asList(120f, 60f, 30f, 15f));
+ // cosine distance vector1= 0.554
+ docs.get(8).addField(vectorField, Arrays.asList(200f, 50f, 100f, 25f));
+ // cosine distance vector1= 0.997
+ docs.get(9).addField(vectorField, Arrays.asList(1.8f, 2.5f, 3.7f, 4.9f));
+ del("*:*");
+ clients.sort(
+ (client1, client2) -> {
+ try {
+ if (client2 instanceof Http2SolrClient && client1 instanceof
HttpSolrClient) {
+ return new URI(((HttpSolrClient) client1).getBaseURL()).getPort()
+ - new URI(((Http2SolrClient)
client2).getBaseURL()).getPort();
+ }
+ } catch (URISyntaxException e) {
+ throw new RuntimeException("Unable to get URI from SolrClient", e);
+ }
+ return 0;
+ });
+ for (SolrInputDocument doc : docs) {
+ indexDoc(doc);
+ }
+ commit();
+ }
+
+ /**
+ * Tests a single lexical query against the Solr server using both combiner
methods.
+ *
+ * @throws Exception if any exception occurs during the test execution
+ */
+ @Test
+ public void testSingleLexicalQuery() throws Exception {
+ prepareIndexDocs();
+ QueryResponse rsp =
+ query(
+ CommonParams.JSON,
+ "{\"queries\":"
+ + "{\"lexical1\":{\"lucene\":{\"query\":\"id:2^10\"}}},"
+ + "\"limit\":5,"
+ + "\"fields\":[\"id\",\"score\",\"title\"],"
+ +
"\"params\":{\"combiner\":true,\"combiner.query\":[\"lexical1\"]}}",
+ CommonParams.QT,
+ "/search");
+ assertEquals(1, rsp.getResults().size());
+ assertFieldValues(rsp.getResults(), id, "2");
+ }
+
+ @Override
+ protected String getShardsString() {
+ if (deadServers == null) return shards;
+ Arrays.sort(shardsArr);
+ StringBuilder sb = new StringBuilder();
+ for (String shard : shardsArr) {
+ if (sb.length() > 0) sb.append(',');
+ sb.append(shard);
+ }
+ return sb.toString();
+ }
+
+ /**
+ * Tests multiple lexical queries using the distributed solr client.
+ *
+ * @throws Exception if any error occurs during the test execution
+ */
+ @Test
+ public void testMultipleLexicalQuery() throws Exception {
+ prepareIndexDocs();
+ String jsonQuery =
+ "{\"queries\":"
+ + "{\"lexical1\":{\"lucene\":{\"query\":\"id:(2^2 OR 3^1 OR 6^2 OR
5^1)\"}},"
+ + "\"lexical2\":{\"lucene\":{\"query\":\"id:(4^1 OR 5^2 OR 7^3 OR
10^2)\"}}},"
+ + "\"limit\":5,"
+ + "\"fields\":[\"id\",\"score\",\"title\"],"
+ +
"\"params\":{\"combiner\":true,\"combiner.query\":[\"lexical1\",\"lexical2\"]}}";
+ QueryResponse rsp = query(CommonParams.JSON, jsonQuery, CommonParams.QT,
"/search");
+ assertEquals(5, rsp.getResults().size());
+ assertFieldValues(rsp.getResults(), id, "5", "7", "2", "6", "3");
+ }
+
+ /**
+ * Test multiple query execution with sort.
+ *
+ * @throws Exception the exception
+ */
+ @Test
+ public void testMultipleQueryWithSort() throws Exception {
+ prepareIndexDocs();
+ String jsonQuery =
+ "{\"queries\":"
+ + "{\"lexical1\":{\"lucene\":{\"query\":\"id:(2^2 OR 3^1 OR 6^2 OR
5^1)\"}},"
+ + "\"lexical2\":{\"lucene\":{\"query\":\"id:(4^1 OR 5^2 OR 7^3 OR
10^2)\"}}},"
+ + "\"limit\":5,\"sort\":\"mod3_idv desc, score desc\""
+ + "\"fields\":[\"id\",\"score\",\"title\"],"
+ +
"\"params\":{\"combiner\":true,\"combiner.query\":[\"lexical1\",\"lexical2\"]}}";
+ QueryResponse rsp = query(CommonParams.JSON, jsonQuery, CommonParams.QT,
"/search");
+ assertEquals(5, rsp.getResults().size());
+ assertFieldValues(rsp.getResults(), id, "5", "2", "7", "10", "4");
+ }
+
+ /**
+ * Tests the hybrid query functionality of the system with various setting
of pagination.
+ *
+ * @throws Exception if any unexpected error occurs during the test
execution.
+ */
+ @Test
+ public void testHybridQueryWithPagination() throws Exception {
+ prepareIndexDocs();
+ QueryResponse rsp =
+ query(
+ CommonParams.JSON,
+ "{\"queries\":"
+ + "{\"lexical1\":{\"lucene\":{\"query\":\"id:(2^2 OR 3^1 OR
6^2 OR 5^1)\"}},"
+ + "\"lexical2\":{\"lucene\":{\"query\":\"id:(4^1 OR 5^2 OR 7^3
OR 10^2)\"}}},"
+ + "\"fields\":[\"id\",\"score\",\"title\"],"
+ +
"\"params\":{\"combiner\":true,\"combiner.query\":[\"lexical1\",\"lexical2\"]}}",
+ CommonParams.QT,
+ "/search");
+ assertFieldValues(rsp.getResults(), id, "5", "7", "2", "6", "3", "10",
"4");
+ rsp =
+ query(
+ CommonParams.JSON,
+ "{\"queries\":"
+ + "{\"lexical1\":{\"lucene\":{\"query\":\"id:(2^2 OR 3^1 OR
6^2 OR 5^1)\"}},"
+ + "\"lexical2\":{\"lucene\":{\"query\":\"id:(4^1 OR 5^2 OR 7^3
OR 10^2)\"}}},"
+ + "\"limit\":4,"
+ + "\"fields\":[\"id\",\"score\",\"title\"],"
+ +
"\"params\":{\"combiner\":true,\"combiner.query\":[\"lexical1\",\"lexical2\"]}}",
+ CommonParams.QT,
+ "/search");
+ assertFieldValues(rsp.getResults(), id, "5", "7", "2", "6");
+ rsp =
+ query(
+ CommonParams.JSON,
+ "{\"queries\":"
+ + "{\"lexical1\":{\"lucene\":{\"query\":\"id:(2^2 OR 3^1 OR
6^2 OR 5^1)\"}},"
+ + "\"lexical2\":{\"lucene\":{\"query\":\"id:(4^1 OR 5^2 OR 7^3
OR 10^2)\"}}},"
+ + "\"limit\":4,\"offset\":3,"
+ + "\"fields\":[\"id\",\"score\",\"title\"],"
+ +
"\"params\":{\"combiner\":true,\"combiner.query\":[\"lexical1\",\"lexical2\"]}}",
+ CommonParams.QT,
+ "/search");
+ assertEquals(4, rsp.getResults().size());
+ assertFieldValues(rsp.getResults(), id, "6", "3", "10", "4");
+ }
+
+ /**
+ * Tests the single query functionality with faceting only.
+ *
+ * @throws Exception if any unexpected error occurs during the test
execution.
+ */
+ @Test
+ public void testQueryWithFaceting() throws Exception {
+ prepareIndexDocs();
+ String jsonQuery =
+ "{\"queries\":"
+ + "{\"lexical\":{\"lucene\":{\"query\":\"id:(2^2 OR 3^1 OR 6^2 OR
5^1)\"}}},"
+ + "\"limit\":3,\"offset\":1"
+ + "\"fields\":[\"id\",\"score\",\"title\"],"
+ +
"\"params\":{\"combiner\":true,\"facet\":true,\"facet.field\":\"mod3_idv\",\"facet.mincount\":1,"
+ + "\"combiner.query\":[\"lexical\"]}}";
+ QueryResponse rsp = query(CommonParams.JSON, jsonQuery, CommonParams.QT,
"/search");
+ assertEquals(3, rsp.getResults().size());
+ assertEquals(4, rsp.getResults().getNumFound());
+ assertEquals("[0 (2), 2 (2)]",
rsp.getFacetFields().get(0).getValues().toString());
+ }
+
+ /**
+ * Tests the combined query feature with faceting and highlighting.
+ *
+ * @throws Exception if any unexpected error occurs during the test
execution.
+ */
+ @Test
+ public void testQueriesWithFacetAndHighlights() throws Exception {
+ prepareIndexDocs();
+ String jsonQuery =
+ "{\"queries\":"
+ + "{\"lexical1\":{\"lucene\":{\"query\":\"id:(2^2 OR 3^1 OR 6^2 OR
5^1)\"}},"
+ + "\"lexical2\":{\"lucene\":{\"query\":\"id:(4^1 OR 5^2 OR 7^3 OR
10^2)\"}}},"
+ + "\"limit\":4,"
+ + "\"fields\":[\"id\",\"score\",\"title\"],"
+ +
"\"params\":{\"combiner\":true,\"facet\":true,\"facet.field\":\"mod3_idv\","
+ + "\"combiner.query\":[\"lexical1\",\"lexical2\"], \"hl\": true,"
+ + "\"hl.fl\": \"title\",\"hl.q\":\"test doc\"}}";
+ QueryResponse rsp = query(CommonParams.JSON, jsonQuery, CommonParams.QT,
"/search");
+ assertEquals(4, rsp.getResults().size());
+ assertFieldValues(rsp.getResults(), id, "5", "7", "2", "6");
+ assertEquals("mod3_idv", rsp.getFacetFields().get(0).getName());
+ assertEquals("[1 (3), 0 (2), 2 (2)]",
rsp.getFacetFields().get(0).getValues().toString());
+ assertEquals(4, rsp.getHighlighting().size());
+ assertEquals(
+ "title <em>test</em> for <em>doc</em> 2",
+ rsp.getHighlighting().get("2").get("title").get(0));
+ assertEquals(
+ "title <em>test</em> for <em>doc</em> 5",
+ rsp.getHighlighting().get("5").get("title").get(0));
+ }
+
+ /**
+ * @see
org.apache.solr.handler.component.CombinedQuerySolrCloudTest#testForcedDistrib()
+ */
+ @Test
+ public void testForcedDistrib() throws Exception {
+ QueryResponse rsp = query("qt", "/forcedDistribTest", "q", "*:*", "rows",
"0");
+ // ForcedDistribSearchHandler would trigger a failure if this didn't work
+ }
+}
diff --git
a/solr/core/src/test/org/apache/solr/handler/component/combine/QueryAndResponseCombinerTest.java
b/solr/core/src/test/org/apache/solr/handler/component/combine/QueryAndResponseCombinerTest.java
new file mode 100644
index 00000000000..aece94c5ea3
--- /dev/null
+++
b/solr/core/src/test/org/apache/solr/handler/component/combine/QueryAndResponseCombinerTest.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.handler.component.combine;
+
+import java.util.List;
+import org.apache.lucene.search.TotalHits;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.search.DocSlice;
+import org.apache.solr.search.QueryResult;
+import org.apache.solr.search.SortedIntDocSet;
+import org.junit.Test;
+
+public class QueryAndResponseCombinerTest extends SolrTestCaseJ4 {
+
+ public static List<QueryResult> getQueryResults() {
+ QueryResult r1 = new QueryResult();
+ r1.setDocList(
+ new DocSlice(
+ 0,
+ 2,
+ new int[] {1, 2},
+ new float[] {0.67f, 0, 0.62f},
+ 3,
+ 0.67f,
+ TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO));
+ r1.setDocSet(new SortedIntDocSet(new int[] {1, 2, 3}, 3));
+ QueryResult r2 = new QueryResult();
+ r2.setDocList(
+ new DocSlice(
+ 0,
+ 1,
+ new int[] {0},
+ new float[] {0.87f},
+ 2,
+ 0.87f,
+ TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO));
+ r2.setDocSet(new SortedIntDocSet(new int[] {0, 1}, 2));
+ return List.of(r1, r2);
+ }
+
+ @Test
+ public void simpleCombine() {
+ QueryResult queryResult =
QueryAndResponseCombiner.simpleCombine(getQueryResults());
+ assertEquals(3, queryResult.getDocList().size());
+ assertEquals(4, queryResult.getDocSet().size());
+ }
+}
diff --git
a/solr/core/src/test/org/apache/solr/handler/component/combine/ReciprocalRankFusionTest.java
b/solr/core/src/test/org/apache/solr/handler/component/combine/ReciprocalRankFusionTest.java
new file mode 100644
index 00000000000..08d684a0d19
--- /dev/null
+++
b/solr/core/src/test/org/apache/solr/handler/component/combine/ReciprocalRankFusionTest.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.handler.component.combine;
+
+import static
org.apache.solr.common.params.CombinerParams.RECIPROCAL_RANK_FUSION;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.CombinerParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.handler.component.ShardDoc;
+import org.apache.solr.search.QueryResult;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * The ReciprocalRankFusionTest class is a unit test suite for the {@link
ReciprocalRankFusion}
+ * class. It verifies the correctness of the fusion algorithm and its
supporting methods.
+ */
+public class ReciprocalRankFusionTest extends SolrTestCaseJ4 {
+
+ public static ReciprocalRankFusion reciprocalRankFusion;
+
+ /**
+ * Initializes the test environment by setting up the {@link
ReciprocalRankFusion} instance with
+ * specific parameters.
+ */
+ @BeforeClass
+ public static void beforeClass() {
+ NamedList<?> args = new NamedList<>(Map.of("k", "20"));
+ reciprocalRankFusion = new ReciprocalRankFusion();
+ reciprocalRankFusion.init(args);
+ }
+
+ /** Tests the functionality of combining the QueryResults across local
search indices. */
+ @Test
+ public void testSimpleCombine() {
+ List<QueryResult> rankedList =
QueryAndResponseCombinerTest.getQueryResults();
+ QueryResult result = QueryAndResponseCombiner.simpleCombine(rankedList);
+ assertEquals(3, result.getDocList().size());
+ assertEquals(4, result.getDocSet().size());
+ }
+
+ /** Test combine docs per queries using RRF. */
+ @Test
+ public void testQueryListCombine() {
+ Map<String, List<ShardDoc>> queriesDocMap = new HashMap<>();
+ ShardDoc shardDoc = new ShardDoc();
+ shardDoc.id = "id1";
+ shardDoc.shard = "shard1";
+ shardDoc.orderInShard = 1;
+ List<ShardDoc> shardDocList = new ArrayList<>();
+ shardDocList.add(shardDoc);
+ shardDoc = new ShardDoc();
+ shardDoc.id = "id2";
+ shardDoc.shard = "shard2";
+ shardDoc.orderInShard = 2;
+ shardDocList.add(shardDoc);
+ queriesDocMap.put(shardDoc.shard, shardDocList);
+
+ shardDoc = new ShardDoc();
+ shardDoc.id = "id2";
+ shardDoc.shard = "shard1";
+ shardDoc.orderInShard = 1;
+ queriesDocMap.put(shardDoc.shard, List.of(shardDoc));
+ SolrParams solrParams = params();
+ assertEquals(20, reciprocalRankFusion.getK());
+ List<ShardDoc> shardDocs = reciprocalRankFusion.combine(queriesDocMap,
solrParams);
+ assertEquals(2, shardDocs.size());
+ assertEquals("id2", shardDocs.get(0).id);
+ }
+
+ @Test
+ public void testImplementationFactory() {
+ Map<String, QueryAndResponseCombiner> combinerMap = new HashMap<>(1);
+ SolrParams emptySolrParms = params();
+ String emptyParamAlgorithm =
+ emptySolrParms.get(CombinerParams.COMBINER_ALGORITHM,
CombinerParams.DEFAULT_COMBINER);
+ assertThrows(
+ SolrException.class,
+ () -> QueryAndResponseCombiner.getImplementation(emptyParamAlgorithm,
combinerMap));
+ SolrParams solrParams = params(CombinerParams.COMBINER_ALGORITHM,
RECIPROCAL_RANK_FUSION);
+ String algorithm =
+ solrParams.get(CombinerParams.COMBINER_ALGORITHM,
CombinerParams.DEFAULT_COMBINER);
+ combinerMap.put(RECIPROCAL_RANK_FUSION, new ReciprocalRankFusion());
+ assertTrue(
+ QueryAndResponseCombiner.getImplementation(algorithm, combinerMap)
+ instanceof ReciprocalRankFusion);
+ }
+}
diff --git
a/solr/core/src/test/org/apache/solr/handler/component/combine/TestCombiner.java
b/solr/core/src/test/org/apache/solr/handler/component/combine/TestCombiner.java
new file mode 100644
index 00000000000..19a87c85839
--- /dev/null
+++
b/solr/core/src/test/org/apache/solr/handler/component/combine/TestCombiner.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.handler.component.combine;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.search.Explanation;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.handler.component.ShardDoc;
+
+/**
+ * The TestCombiner class is an extension of QueryAndResponseCombiner that
implements custom logic
+ * for combining ranked lists using linear sorting of score from all rank
lists. This is just for
+ * testing purpose which has been used in test suite
CombinedQueryComponentTest for e2e testing of
+ * Plugin based Combiner approach.
+ */
+public class TestCombiner extends QueryAndResponseCombiner {
+
+ private int testInt;
+
+ public int getTestInt() {
+ return testInt;
+ }
+
+ @Override
+ public void init(NamedList<?> args) {
+ Object kParam = args.get("var1");
+ if (kParam != null) {
+ this.testInt = Integer.parseInt(kParam.toString());
+ }
+ }
+
+ @Override
+ public List<ShardDoc> combine(Map<String, List<ShardDoc>> shardDocMap,
SolrParams solrParams) {
+ return List.of();
+ }
+
+ @Override
+ public SimpleOrderedMap<Explanation> getExplanations(
+ String[] queryKeys,
+ Map<String, List<ShardDoc>> queriesDocMap,
+ List<ShardDoc> combinedQueriesDocs,
+ SolrParams solrParams) {
+ SimpleOrderedMap<Explanation> docIdsExplanations = new
SimpleOrderedMap<>();
+ docIdsExplanations.add("combinerDetails", Explanation.match(testInt, "this
is test combiner"));
+ return docIdsExplanations;
+ }
+}
diff --git
a/solr/solr-ref-guide/modules/query-guide/pages/json-combined-query-dsl.adoc
b/solr/solr-ref-guide/modules/query-guide/pages/json-combined-query-dsl.adoc
new file mode 100644
index 00000000000..98d3c2e96b8
--- /dev/null
+++ b/solr/solr-ref-guide/modules/query-guide/pages/json-combined-query-dsl.adoc
@@ -0,0 +1,107 @@
+= JSON Combined Query DSL
+:tabs-sync-option:
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF 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.
+
+The Combined Query feature aims to execute multiple queries of multiple kinds
across multiple shards of a collection and combine their result basis an
algorithm (like Reciprocal Rank Fusion).
+It is extending JSON Query DSL ultimately enabling Hybrid Search.
+
+[NOTE]
+====
+This feature is currently unsupported for grouping and Cursors.
+====
+
+== Query DSL Structure
+The query structure is similar to JSON Query DSL except for how multiple
queries are defined along with their parameters.
+
+* Multiple queries can be defined under the `queries` key by providing their
name with the same syntax as a single query is defined with the key `query`.
+* In addition to the other supported parameters, there are several parameters
which can be defined under `params` key as below:
+`combiner` | Default: `false`::
+ Enables the combined query mode when set to `true`.
+`combiner.query`::
+ The list of queries to be executed as defined in the `queries` key.
Example: `["query1", "query2"]`
+`combiner.algorithm` | Default: `rrf`::
+ The algorithm to be used for combining the results. Reciprocal Rank Fusion
(RRF) is the in-built fusion algorithm.
+ Any other algorithm can be configured using
xref:json-combined-query-dsl.adoc#combiner-algorithm-plugin[plugin].
+`combiner.rrf.k` | Default: `60`::
+ The k parameter in the RRF algorithm.
+
+=== Example
+
+Below is a sample JSON query payload:
+
+```
+{
+ "queries": {
+ "lexical1": {
+ "lucene": {
+ "query": "title:sales"
+ }
+ },
+ "vector": {
+ "knn": {
+ "f": "vector",
+ "topK" :5,
+ "query": "[0.1,-0.34,0.89,0.02]"
+ }
+ }
+ },
+ "limit": 5,
+ "fields": ["id", "score", "title"],
+ "params": {
+ "combiner": true,
+ "combiner.query": ["lexical1", "vector"],
+ "combiner.algorithm": "rrf",
+ "combiner.rrf.k": "15"
+ }
+}
+```
+
+== Search Handler Configuration
+
+Combined Query Feature has a separate handler with class
`solr.CombinedQuerySearchHandler` which can be configured as below:
+
+```
+<requestHandler name="/search" class="solr.CombinedQuerySearchHandler">
+.....
+</requestHandler>
+```
+
+The Search Handler also accepts parameters as below:
+
+`maxCombinerQueries`::
+ This parameter can be set to put upper limit check on the maximum number of
queries can be executed defined in `combiner.query`.
+ It defaults to `5` if not set.
+
+=== Combiner Algorithm Plugin
+
+As mentioned xref:json-combined-query-dsl.adoc#query-dsl-structure[above],
custom algorithms can be configured to combine the results across multiple
queries.
+The Combined Query Search Handler definition takes parameter `combiners` where
a custom class can be used to define the algorithm by giving a name and the
parameters required.
+
+Example of the Search Handler as below:
+```
+<searchComponent class="solr.CombinedQueryComponent" name="combined_query">
+ <int name="maxCombinerQueries">2</int>
+ <lst name="combiners">
+ <lst name="customAlgorithm">
+ <str
name="class">org.apache.solr.search.combine.CustomCombiner</str>
+ <int name="var1">35</int>
+ <str name="var2">customValue</str>
+ </lst>
+ </lst>
+ </searchComponent>
+```
diff --git a/solr/solr-ref-guide/modules/query-guide/querying-nav.adoc
b/solr/solr-ref-guide/modules/query-guide/querying-nav.adoc
index 9890f089047..ecdd4603dcf 100644
--- a/solr/solr-ref-guide/modules/query-guide/querying-nav.adoc
+++ b/solr/solr-ref-guide/modules/query-guide/querying-nav.adoc
@@ -27,6 +27,7 @@
** xref:local-params.adoc[]
** xref:json-request-api.adoc[]
*** xref:json-query-dsl.adoc[]
+*** xref:json-combined-query-dsl.adoc[]
** xref:searching-nested-documents.adoc[]
** xref:block-join-query-parser.adoc[]
** xref:join-query-parser.adoc[]
diff --git
a/solr/solrj/src/java/org/apache/solr/common/params/CombinerParams.java
b/solr/solrj/src/java/org/apache/solr/common/params/CombinerParams.java
new file mode 100644
index 00000000000..23d735d595d
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/common/params/CombinerParams.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.common.params;
+
+/**
+ * This class provides constants for configuration parameters related to the
combiner. It defines
+ * keys for various properties used in the combiner configuration.
+ */
+public class CombinerParams {
+
+ private CombinerParams() {}
+
+ public static final String COMBINER = "combiner";
+ public static final String COMBINER_ALGORITHM = COMBINER + ".algorithm";
+ public static final String COMBINER_QUERY = COMBINER + ".query";
+ public static final String RECIPROCAL_RANK_FUSION = "rrf";
+ public static final String COMBINER_RRF_K = COMBINER + "." +
RECIPROCAL_RANK_FUSION + ".k";
+ public static final String DEFAULT_COMBINER = RECIPROCAL_RANK_FUSION;
+ public static final int DEFAULT_COMBINER_RRF_K = 60;
+ public static final int DEFAULT_MAX_COMBINER_QUERIES = 5;
+}