You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by dw...@apache.org on 2021/03/10 09:53:23 UTC

[lucene] 09/45: SOLR-15038: Add elevateOnlyDocsMatchingQuery and collectElevatedDocsWhenCollapsing parameters to query elevation.

This is an automated email from the ASF dual-hosted git repository.

dweiss pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 05e2a91a2f27d2f121e5ff04b16796aadd91d8c3
Author: Tobias Kaessmann <to...@otto.de>
AuthorDate: Wed Feb 17 09:47:03 2021 +0100

    SOLR-15038: Add elevateOnlyDocsMatchingQuery and collectElevatedDocsWhenCollapsing parameters to query elevation.
---
 solr/CHANGES.txt                                   |   5 +-
 .../handler/component/QueryElevationComponent.java |   5 +-
 .../solr/search/CollapsingQParserPlugin.java       |  92 +++++++++++++------
 .../component/QueryElevationComponentTest.java     | 101 +++++++++++++++++++++
 solr/server/etc/security.policy                    |   2 +-
 .../src/collapse-and-expand-results.adoc           |   4 +
 .../src/the-query-elevation-component.adoc         |   4 +
 .../solr/common/params/QueryElevationParams.java   |   7 ++
 8 files changed, 187 insertions(+), 33 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 86d12cc..2d00b81 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -14,7 +14,6 @@ New Features
 
 * SOLR-15150: New update.partial.requireInPlace=true option to prevent any partial document updates that can't be done In-Place (hossman)
 
-
 Improvements
 ---------------------
 * SOLR-15081: Metrics for a core: add SolrCloud "isLeader" and "replicaState".  (David Smiley)
@@ -30,6 +29,9 @@ Improvements
 
 * SOLR-15101: Add "list" and "delete" APIs for managing incremental backups (Jason Gerlowski, shalin, Cao Manh Dat)
 
+* SOLR-15038: Add elevateOnlyDocsMatchingQuery and collectElevatedDocsWhenCollapsing parameters to query elevation.
+  (Dennis Berger, Tobias Kässmann via Bruno Roustant)
+
 Optimizations
 ---------------------
 * SOLR-15079: Block Collapse - Faster collapse code when groups are co-located via Block Join style nested doc indexing.
@@ -42,7 +44,6 @@ Bug Fixes
 * SOLR-15078: Fix ExpandComponent behavior when expanding on numeric fields to differentiate '0' group from null group (hossman)
 * SOLR-15149: Better exception handling for LTR model creation errors (Alessandro Benedetti, Christine Poerschke)
 
-
 Other Changes
 ---------------------
 * SOLR-15118: Deprecate CollectionAdminRequest.getV2Request(). (Jason Gerlowski)
diff --git a/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java b/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java
index 79b8d1b..6e66d80 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java
@@ -509,7 +509,10 @@ public class QueryElevationComponent extends SearchComponent implements SolrCore
       rb.setQuery(new BoostQuery(elevation.includeQuery, 0f));
     } else {
       BooleanQuery.Builder queryBuilder = new BooleanQuery.Builder();
-      queryBuilder.add(rb.getQuery(), BooleanClause.Occur.SHOULD);
+      BooleanClause.Occur queryOccurrence =
+              params.getBool(QueryElevationParams.ELEVATE_ONLY_DOCS_MATCHING_QUERY, false) ?
+                      BooleanClause.Occur.MUST : BooleanClause.Occur.SHOULD;
+      queryBuilder.add(rb.getQuery(), queryOccurrence);
       queryBuilder.add(new BoostQuery(elevation.includeQuery, 0f), BooleanClause.Occur.SHOULD);
       if (elevation.excludeQueries != null) {
         if (params.getBool(QueryElevationParams.MARK_EXCLUDES, false)) {
diff --git a/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
index df6a520..09d4c55 100644
--- a/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
+++ b/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
@@ -147,6 +147,12 @@ public class CollapsingQParserPlugin extends QParserPlugin {
   public static final String HINT_BLOCK = "block";
 
   /**
+   * If elevation is used in combination with the collapse query parser, we can define that we only want to return the
+   * representative and not all elevated docs by setting this parameter to false (true by default).
+   */
+  public static String COLLECT_ELEVATED_DOCS_WHEN_COLLAPSING = "collectElevatedDocsWhenCollapsing";
+
+  /**
    * @deprecated use {@link NullPolicy} instead.
    */
   @Deprecated
@@ -583,6 +589,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
     private int nullPolicy;
     private float nullScore = -Float.MAX_VALUE;
     private int nullDoc = -1;
+    private boolean collectElevatedDocsWhenCollapsing;
     private FloatArrayList nullScores;
 
     private final BoostedDocsCollector boostedDocsCollector;
@@ -592,9 +599,11 @@ public class CollapsingQParserPlugin extends QParserPlugin {
                              DocValuesProducer collapseValuesProducer,
                              int nullPolicy,
                              IntIntHashMap boostDocsMap,
-                             IndexSearcher searcher) throws IOException {
+                             IndexSearcher searcher,
+                             boolean collectElevatedDocsWhenCollapsing) throws IOException {
       this.maxDoc = maxDoc;
       this.contexts = new LeafReaderContext[segments];
+      this.collectElevatedDocsWhenCollapsing = collectElevatedDocsWhenCollapsing;
       List<LeafReaderContext> con = searcher.getTopReaderContext().leaves();
       for(int i=0; i<con.size(); i++) {
         contexts[i] = con.get(i);
@@ -651,12 +660,14 @@ public class CollapsingQParserPlugin extends QParserPlugin {
           ord = -1;
         }
       }
-      
-      // Check to see if we have documents boosted by the QueryElevationComponent
-      if (0 <= ord) {
-        if (boostedDocsCollector.collectIfBoosted(ord, globalDoc)) return;
-      } else {
-        if (boostedDocsCollector.collectInNullGroupIfBoosted(globalDoc)) return;
+
+      if (collectElevatedDocsWhenCollapsing) {
+        // Check to see if we have documents boosted by the QueryElevationComponent
+        if (0 <= ord) {
+          if (boostedDocsCollector.collectIfBoosted(ord, globalDoc)) return;
+        } else {
+          if (boostedDocsCollector.collectInNullGroupIfBoosted(globalDoc)) return;
+        }
       }
 
       if(ord > -1) {
@@ -783,6 +794,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
     private int nullDoc = -1;
     private FloatArrayList nullScores;
     private String field;
+    private boolean collectElevatedDocsWhenCollapsing;
 
     private final BoostedDocsCollector boostedDocsCollector;
     
@@ -792,9 +804,11 @@ public class CollapsingQParserPlugin extends QParserPlugin {
                              int size,
                              String field,
                              IntIntHashMap boostDocsMap,
-                             IndexSearcher searcher) {
+                             IndexSearcher searcher,
+                             boolean collectElevatedDocsWhenCollapsing) {
       this.maxDoc = maxDoc;
       this.contexts = new LeafReaderContext[segments];
+      this.collectElevatedDocsWhenCollapsing = collectElevatedDocsWhenCollapsing;
       List<LeafReaderContext> con = searcher.getTopReaderContext().leaves();
       for(int i=0; i<con.size(); i++) {
         contexts[i] = con.get(i);
@@ -825,8 +839,11 @@ public class CollapsingQParserPlugin extends QParserPlugin {
       final int globalDoc = docBase+contextDoc;
       if (collapseValues.advanceExact(contextDoc)) {
         final int collapseValue = (int) collapseValues.longValue();
-        // Check to see if we have documents boosted by the QueryElevationComponent (skip normal strategy based collection)
-        if (boostedDocsCollector.collectIfBoosted(collapseValue, globalDoc)) return;
+
+        if (collectElevatedDocsWhenCollapsing) {
+          // Check to see if we have documents boosted by the QueryElevationComponent (skip normal strategy based collection)
+          if (boostedDocsCollector.collectIfBoosted(collapseValue, globalDoc)) return;
+        }
 
         float score = scorer.score();
         final int idx;
@@ -845,9 +862,11 @@ public class CollapsingQParserPlugin extends QParserPlugin {
         }
 
       } else { // Null Group...
-        
-        // Check to see if we have documents boosted by the QueryElevationComponent (skip normal strategy based collection)
-        if (boostedDocsCollector.collectInNullGroupIfBoosted(globalDoc)) return;
+
+        if (collectElevatedDocsWhenCollapsing){
+          // Check to see if we have documents boosted by the QueryElevationComponent (skip normal strategy based collection)
+          if (boostedDocsCollector.collectInNullGroupIfBoosted(globalDoc)) return;
+        }
 
         if(nullPolicy == NullPolicy.COLLAPSE.getCode()) {
           float score = scorer.score();
@@ -956,7 +975,9 @@ public class CollapsingQParserPlugin extends QParserPlugin {
     private OrdFieldValueStrategy collapseStrategy;
     private boolean needsScores4Collapsing;
     private boolean needsScores;
-    
+
+    private boolean collectElevatedDocsWhenCollapsing;
+
     private final BoostedDocsCollector boostedDocsCollector;
 
     public OrdFieldValueCollector(int maxDoc,
@@ -969,10 +990,12 @@ public class CollapsingQParserPlugin extends QParserPlugin {
                                   boolean needsScores,
                                   FieldType fieldType,
                                   IntIntHashMap boostDocsMap,
-                                  FunctionQuery funcQuery, IndexSearcher searcher) throws IOException{
+                                  FunctionQuery funcQuery, IndexSearcher searcher,
+                                  boolean collectElevatedDocsWhenCollapsing) throws IOException{
 
       assert ! GroupHeadSelectorType.SCORE.equals(groupHeadSelector.type);
 
+      this.collectElevatedDocsWhenCollapsing = collectElevatedDocsWhenCollapsing;
       this.maxDoc = maxDoc;
       this.contexts = new LeafReaderContext[segments];
       List<LeafReaderContext> con = searcher.getTopReaderContext().leaves();
@@ -1051,12 +1074,14 @@ public class CollapsingQParserPlugin extends QParserPlugin {
           ord = segmentValues.ordValue();
         }
       }
-      
-      // Check to see if we have documents boosted by the QueryElevationComponent (skip normal strategy based collection)
-      if (-1 == ord) {
-        if (boostedDocsCollector.collectInNullGroupIfBoosted(globalDoc)) return;
-      } else {
-        if (boostedDocsCollector.collectIfBoosted(ord, globalDoc)) return;
+
+      if (collectElevatedDocsWhenCollapsing){
+        // Check to see if we have documents boosted by the QueryElevationComponent (skip normal strategy based collection)
+        if (-1 == ord) {
+          if (boostedDocsCollector.collectInNullGroupIfBoosted(globalDoc)) return;
+        } else {
+          if (boostedDocsCollector.collectIfBoosted(ord, globalDoc)) return;
+        }
       }
       
       collapseStrategy.collapse(ord, contextDoc, globalDoc);
@@ -1164,6 +1189,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
     private String collapseField;
     
     private final BoostedDocsCollector boostedDocsCollector;
+    private boolean collectElevatedDocsWhenCollapsing;
 
     public IntFieldValueCollector(int maxDoc,
                                   int size,
@@ -1177,7 +1203,9 @@ public class CollapsingQParserPlugin extends QParserPlugin {
                                   FieldType fieldType,
                                   IntIntHashMap boostDocsMap,
                                   FunctionQuery funcQuery,
-                                  IndexSearcher searcher) throws IOException{
+                                  IndexSearcher searcher,
+                                  boolean collectElevatedDocsWhenCollapsing) throws IOException{
+      this.collectElevatedDocsWhenCollapsing = collectElevatedDocsWhenCollapsing;
 
       assert ! GroupHeadSelectorType.SCORE.equals(groupHeadSelector.type);
 
@@ -1241,10 +1269,11 @@ public class CollapsingQParserPlugin extends QParserPlugin {
         collapseStrategy.collapse(collapseKey, contextDoc, globalDoc);
         
       } else { // Null Group...
-        
-        // Check to see if we have documents boosted by the QueryElevationComponent (skip normal strategy based collection)
-        if (boostedDocsCollector.collectInNullGroupIfBoosted(globalDoc)) return;
 
+        if (collectElevatedDocsWhenCollapsing){
+          // Check to see if we have documents boosted by the QueryElevationComponent (skip normal strategy based collection)
+          if (boostedDocsCollector.collectInNullGroupIfBoosted(globalDoc)) return;
+        }
         if (NullPolicy.IGNORE.getCode() != nullPolicy) {
           collapseStrategy.collapseNullGroup(contextDoc, globalDoc);
         }
@@ -1892,20 +1921,23 @@ public class CollapsingQParserPlugin extends QParserPlugin {
       int maxDoc = searcher.maxDoc();
       int leafCount = searcher.getTopReaderContext().leaves().size();
 
+      SolrRequestInfo req = SolrRequestInfo.getRequestInfo();
+      boolean collectElevatedDocsWhenCollapsing = req != null && req.getReq().getParams().getBool(COLLECT_ELEVATED_DOCS_WHEN_COLLAPSING, true);
+
       if (GroupHeadSelectorType.SCORE.equals(groupHeadSelector.type)) {
 
         if (collapseFieldType instanceof StrField) {
           if (blockCollapse) {
             return new BlockOrdScoreCollector(collapseField, nullPolicy, boostDocs);
           }
-          return new OrdScoreCollector(maxDoc, leafCount, docValuesProducer, nullPolicy, boostDocs, searcher);
+          return new OrdScoreCollector(maxDoc, leafCount, docValuesProducer, nullPolicy, boostDocs, searcher, collectElevatedDocsWhenCollapsing);
 
         } else if (isNumericCollapsible(collapseFieldType)) {
           if (blockCollapse) {
             return new BlockIntScoreCollector(collapseField, nullPolicy, boostDocs);
           }
 
-          return new IntScoreCollector(maxDoc, leafCount, nullPolicy, size, collapseField, boostDocs, searcher);
+          return new IntScoreCollector(maxDoc, leafCount, nullPolicy, size, collapseField, boostDocs, searcher, collectElevatedDocsWhenCollapsing);
 
         } else {
           throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
@@ -1935,7 +1967,8 @@ public class CollapsingQParserPlugin extends QParserPlugin {
                                             minMaxFieldType,
                                             boostDocs,
                                             funcQuery,
-                                            searcher);
+                                            searcher,
+                                            collectElevatedDocsWhenCollapsing);
 
         } else if (isNumericCollapsible(collapseFieldType)) {
 
@@ -1960,7 +1993,8 @@ public class CollapsingQParserPlugin extends QParserPlugin {
                                             minMaxFieldType,
                                             boostDocs,
                                             funcQuery,
-                                            searcher);
+                                            searcher,
+                                            collectElevatedDocsWhenCollapsing);
         } else {
           throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
               "Collapsing field should be of either String, Int or Float type");
diff --git a/solr/core/src/test/org/apache/solr/handler/component/QueryElevationComponentTest.java b/solr/core/src/test/org/apache/solr/handler/component/QueryElevationComponentTest.java
index 3977efc..7f57d39 100644
--- a/solr/core/src/test/org/apache/solr/handler/component/QueryElevationComponentTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/component/QueryElevationComponentTest.java
@@ -37,6 +37,7 @@ import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.search.CollapsingQParserPlugin;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.util.FileUtils;
 import org.junit.Before;
@@ -931,6 +932,106 @@ public class QueryElevationComponentTest extends SolrTestCaseJ4 {
     }
   }
 
+  @Test
+  public void testOnlyDocsInSearchResultsWillBeElevated() throws Exception {
+    try {
+      init("schema12.xml");
+      assertU(adoc("id", "1", "title", "XXXX", "str_s1", "a"));
+      assertU(adoc("id", "2", "title", "YYYY", "str_s1", "b"));
+      assertU(adoc("id", "3", "title", "ZZZZ", "str_s1", "c"));
+
+      assertU(adoc("id", "4", "title", "XXXX XXXX", "str_s1", "x"));
+      assertU(adoc("id", "5", "title", "YYYY YYYY", "str_s1", "y"));
+      assertU(adoc("id", "6", "title", "XXXX XXXX", "str_s1", "z"));
+      assertU(adoc("id", "7", "title", "AAAA", "str_s1", "a"));
+
+      assertU(commit());
+
+      // default behaviour
+      assertQ("", req(
+              CommonParams.Q, "YYYY",
+              CommonParams.QT, "/elevate",
+              QueryElevationParams.ELEVATE_ONLY_DOCS_MATCHING_QUERY, "false",
+              CommonParams.FL, "id, score, [elevated]"),
+              "//*[@numFound='3']",
+              "//result/doc[1]/str[@name='id'][.='1']",
+              "//result/doc[2]/str[@name='id'][.='2']",
+              "//result/doc[3]/str[@name='id'][.='5']",
+              "//result/doc[1]/bool[@name='[elevated]'][.='true']",
+              "//result/doc[2]/bool[@name='[elevated]'][.='true']",
+              "//result/doc[3]/bool[@name='[elevated]'][.='false']"
+      );
+
+      // only docs that matches q
+      assertQ("", req(
+              CommonParams.Q, "YYYY",
+              CommonParams.QT, "/elevate",
+              QueryElevationParams.ELEVATE_ONLY_DOCS_MATCHING_QUERY, "true",
+              CommonParams.FL, "id, score, [elevated]"),
+              "//*[@numFound='2']",
+              "//result/doc[1]/str[@name='id'][.='2']",
+              "//result/doc[2]/str[@name='id'][.='5']",
+              "//result/doc[1]/bool[@name='[elevated]'][.='true']",
+              "//result/doc[2]/bool[@name='[elevated]'][.='false']"
+      );
+
+
+
+    } finally {
+      delete();
+    }
+  }
+
+  @Test
+  public void testOnlyRepresentativeIsVisibleWhenCollapsing() throws Exception {
+    try {
+      init("schema12.xml");
+      assertU(adoc("id", "1", "title", "ZZZZ", "str_s1", "a"));
+      assertU(adoc("id", "2", "title", "ZZZZ", "str_s1", "b"));
+      assertU(adoc("id", "3", "title", "ZZZZ ZZZZ", "str_s1", "a"));
+      assertU(adoc("id", "4", "title", "ZZZZ ZZZZ", "str_s1", "c"));
+
+      assertU(commit());
+
+      // default behaviour - all elevated docs are visible
+      assertQ("", req(
+              CommonParams.Q, "ZZZZ",
+              CommonParams.QT, "/elevate",
+              CollapsingQParserPlugin.COLLECT_ELEVATED_DOCS_WHEN_COLLAPSING, "true",
+              CommonParams.FQ, "{!collapse field=str_s1 sort='score desc'}",
+              CommonParams.FL, "id, score, [elevated]"),
+              "//*[@numFound='4']",
+              "//result/doc[1]/str[@name='id'][.='1']",
+              "//result/doc[2]/str[@name='id'][.='2']",
+              "//result/doc[3]/str[@name='id'][.='3']",
+              "//result/doc[4]/str[@name='id'][.='4']",
+              "//result/doc[1]/bool[@name='[elevated]'][.='true']",
+              "//result/doc[2]/bool[@name='[elevated]'][.='true']",
+              "//result/doc[3]/bool[@name='[elevated]'][.='true']",
+              "//result/doc[4]/bool[@name='[elevated]'][.='false']"
+      );
+
+      // only representative elevated doc visible
+      assertQ("", req(
+              CommonParams.Q, "ZZZZ",
+              CommonParams.QT, "/elevate",
+              CollapsingQParserPlugin.COLLECT_ELEVATED_DOCS_WHEN_COLLAPSING, "false",
+              CommonParams.FQ, "{!collapse field=str_s1 sort='score desc'}",
+              CommonParams.FL, "id, score, [elevated]"),
+              "//*[@numFound='3']",
+              "//result/doc[1]/str[@name='id'][.='2']",
+              "//result/doc[2]/str[@name='id'][.='3']",
+              "//result/doc[3]/str[@name='id'][.='4']",
+              "//result/doc[1]/bool[@name='[elevated]'][.='true']",
+              "//result/doc[2]/bool[@name='[elevated]'][.='true']",
+              "//result/doc[3]/bool[@name='[elevated]'][.='false']"
+      );
+
+    } finally {
+      delete();
+    }
+  }
+
   private static Set<BytesRef> toIdSet(String... ids) {
     return Arrays.stream(ids).map(BytesRef::new).collect(Collectors.toSet());
   }
diff --git a/solr/server/etc/security.policy b/solr/server/etc/security.policy
index bcf82b9..9f687f2 100644
--- a/solr/server/etc/security.policy
+++ b/solr/server/etc/security.policy
@@ -23,7 +23,7 @@ grant {
   // contain read access to only what we need:
   // 3rd party jar resources (where symlinks are not supported), test-files/ resources
   permission java.io.FilePermission "${common.dir}${/}-", "read";
-  permission java.io.FilePermission "${common.dir}${/}..${/}solr${/}-", "read";
+  permission java.io.FilePermission "${common.dir}${/}..${/}solr${/}-", "read,write";
   // 3rd party jar resources (where symlinks are supported)
   permission java.io.FilePermission "${user.home}${/}.ivy2${/}cache${/}-", "read";
   // system jar resources
diff --git a/solr/solr-ref-guide/src/collapse-and-expand-results.adoc b/solr/solr-ref-guide/src/collapse-and-expand-results.adoc
index f3846c1..379136a 100644
--- a/solr/solr-ref-guide/src/collapse-and-expand-results.adoc
+++ b/solr/solr-ref-guide/src/collapse-and-expand-results.adoc
@@ -81,6 +81,10 @@ The data structures used for collapsing grow dynamically when collapsing on nume
 +
 The default is 100,000.
 
+`collectElevatedDocsWhenCollapsing`::
+In combination with the <<collapse-and-expand-results.adoc#collapsing-query-parser,Collapse Query Parser>> all elevated docs are visible at the beginning of the result set.
+If this parameter is `false`, only the representative is visible if the elevated docs has the same collapse key (default is `true`).
+
 
 === Sample Usage Syntax
 
diff --git a/solr/solr-ref-guide/src/the-query-elevation-component.adoc b/solr/solr-ref-guide/src/the-query-elevation-component.adoc
index f43a73a..75b187b 100644
--- a/solr/solr-ref-guide/src/the-query-elevation-component.adoc
+++ b/solr/solr-ref-guide/src/the-query-elevation-component.adoc
@@ -93,6 +93,10 @@ they be subject to whatever the sort criteria is?  True by default.
 This is also a request parameter, which will override the config.
 The effect is most apparent when forceElevation is true and there is sorting on fields.
 
+`elevateOnlyDocsMatchingQuery`::
+By default, the component will also elevate docs that aren't part of the search result (matching the query).
+If you only want to elevate the docs that are part of the search result, set this to `true` (default is `false`).
+
 === The elevate.xml File
 
 Elevated query results can be configured in an external XML file specified in the `config-file` argument. An `elevate.xml` file might look like this:
diff --git a/solr/solrj/src/java/org/apache/solr/common/params/QueryElevationParams.java b/solr/solrj/src/java/org/apache/solr/common/params/QueryElevationParams.java
index 9a31f69..e77408e 100644
--- a/solr/solrj/src/java/org/apache/solr/common/params/QueryElevationParams.java
+++ b/solr/solrj/src/java/org/apache/solr/common/params/QueryElevationParams.java
@@ -55,4 +55,11 @@ public interface QueryElevationParams {
    * they be subject to whatever the sort criteria is?  True by default.
    */
   String USE_CONFIGURED_ELEVATED_ORDER = "useConfiguredElevatedOrder";
+
+  /**
+   * By default, the component will also elevate docs that aren't part of the search result (matching the query).
+   * If you only want to elevate the docs that are part of the search result, set this to true. False by default.
+   */
+  String ELEVATE_ONLY_DOCS_MATCHING_QUERY = "elevateOnlyDocsMatchingQuery";
+
 }
\ No newline at end of file