You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@solr.apache.org by ho...@apache.org on 2024/02/08 23:32:33 UTC

(solr) branch main updated: SOLR-16858: KnnQParser's "Pre-Filtering" behavior is now controlable via local params (closes #2157)

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

hossman pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/solr.git


The following commit(s) were added to refs/heads/main by this push:
     new b88fa0d7d15 SOLR-16858: KnnQParser's "Pre-Filtering" behavior is now controlable via local params (closes #2157)
b88fa0d7d15 is described below

commit b88fa0d7d15d70cea669218ab9dfd1068f9fcf73
Author: Chris Hostetter <ho...@apache.org>
AuthorDate: Thu Feb 8 16:32:05 2024 -0700

    SOLR-16858: KnnQParser's "Pre-Filtering" behavior is now controlable via local params (closes #2157)
---
 solr/CHANGES.txt                                   |   2 +
 .../org/apache/solr/search/neural/KnnQParser.java  | 146 ++++++-
 .../org/apache/solr/search/QueryEqualityTest.java  |  90 ++++-
 .../apache/solr/search/neural/KnnQParserTest.java  | 419 ++++++++++++++++++++-
 .../query-guide/pages/dense-vector-search.adoc     | 114 ++++--
 5 files changed, 715 insertions(+), 56 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index cc2ee02252f..54b641873d6 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -95,6 +95,8 @@ Improvements
 
 * SOLR-17146: Add DelegatingBackupRepository and alternative checksum verification (Bruno Roustant)
 
+* SOLR-16858: KnnQParser's "Pre-Filtering" behavior is now controlable via local params (hossman)
+
 Optimizations
 ---------------------
 * SOLR-17144: Close searcherExecutor thread per core after 1 minute (Pierre Salagnac, Christine Poerschke)
diff --git a/solr/core/src/java/org/apache/solr/search/neural/KnnQParser.java b/solr/core/src/java/org/apache/solr/search/neural/KnnQParser.java
index 3d8cd64a5e2..252a4fcabc7 100644
--- a/solr/core/src/java/org/apache/solr/search/neural/KnnQParser.java
+++ b/solr/core/src/java/org/apache/solr/search/neural/KnnQParser.java
@@ -17,11 +17,13 @@
 package org.apache.solr.search.neural;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import org.apache.lucene.search.Query;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.schema.DenseVectorField;
 import org.apache.solr.schema.FieldType;
@@ -29,11 +31,14 @@ import org.apache.solr.schema.SchemaField;
 import org.apache.solr.search.QParser;
 import org.apache.solr.search.QueryParsing;
 import org.apache.solr.search.QueryUtils;
-import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.search.SyntaxError;
 
 public class KnnQParser extends QParser {
 
+  static final String PRE_FILTER = "preFilter";
+  static final String EXCLUDE_TAGS = "excludeTags";
+  static final String INCLUDE_TAGS = "includeTags";
+
   // retrieve the top K results based on the distance similarity function
   static final String TOP_K = "topK";
   static final int DEFAULT_TOP_K = 10;
@@ -82,20 +87,133 @@ public class KnnQParser extends QParser {
   }
 
   private Query getFilterQuery() throws SolrException, SyntaxError {
-    boolean isSubQuery = recurseCount != 0;
-    if (!isFilter() && !isSubQuery) {
-      String[] filterQueries = req.getParams().getParams(CommonParams.FQ);
-      if (filterQueries != null && filterQueries.length != 0) {
-        try {
-          List<Query> filters = QueryUtils.parseFilterQueries(req);
-          SolrIndexSearcher.ProcessedFilter processedFilter =
-              req.getSearcher().getProcessedFilter(filters);
-          return processedFilter.filter;
-        } catch (IOException e) {
-          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+
+    // Default behavior of FQ wrapping, and suitability of some local params
+    // depends on wether we are a sub-query or not
+    final boolean isSubQuery = recurseCount != 0;
+
+    // include/exclude tags for global fqs to wrap;
+    // Check these up front for error handling if combined with `fq` local param.
+    final List<String> includedGlobalFQTags = getLocalParamTags(INCLUDE_TAGS);
+    final List<String> excludedGlobalFQTags = getLocalParamTags(EXCLUDE_TAGS);
+    final boolean haveGlobalFQTags =
+        !(includedGlobalFQTags.isEmpty() && excludedGlobalFQTags.isEmpty());
+
+    if (haveGlobalFQTags) {
+      // Some early error handling of incompatible options...
+
+      if (isFilter()) { // this knn query is itself a filter query
+        throw new SolrException(
+            SolrException.ErrorCode.BAD_REQUEST,
+            "Knn Query Parser used as a filter does not support "
+                + INCLUDE_TAGS
+                + " or "
+                + EXCLUDE_TAGS
+                + " localparams");
+      }
+
+      if (isSubQuery) { // this knn query is a sub-query of a broader query (possibly disjunction)
+        throw new SolrException(
+            SolrException.ErrorCode.BAD_REQUEST,
+            "Knn Query Parser used as a sub-query does not support "
+                + INCLUDE_TAGS
+                + " or "
+                + EXCLUDE_TAGS
+                + " localparams");
+      }
+    }
+
+    // Explicit local params specifying the filter(s) to wrap
+    final String[] preFilters = getLocalParams().getParams(PRE_FILTER);
+    if (null != preFilters) {
+
+      // We don't particularly care if preFilters is empty, the usage below will still work,
+      // but SolrParams API says it should be null not empty...
+      assert 0 != preFilters.length
+          : "SolrParams.getParams should return null, never zero len array";
+
+      if (haveGlobalFQTags) {
+        throw new SolrException(
+            SolrException.ErrorCode.BAD_REQUEST,
+            "Knn Query Parser does not support combining "
+                + PRE_FILTER
+                + " localparam with either "
+                + INCLUDE_TAGS
+                + " or "
+                + EXCLUDE_TAGS
+                + " localparams");
+      }
+
+      final List<Query> preFilterQueries = new ArrayList<>(preFilters.length);
+      for (String f : preFilters) {
+        final QParser parser = subQuery(f, null);
+        parser.setIsFilter(true);
+
+        // maybe null, ie: `preFilter=""`
+        final Query filter = parser.getQuery();
+        if (null != filter) {
+          preFilterQueries.add(filter);
         }
       }
+      try {
+        return req.getSearcher().getProcessedFilter(preFilterQueries).filter;
+      } catch (IOException e) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+      }
+    }
+
+    // No explicit `preFilter` localparams specifying what we should filter on.
+    //
+    // So now, if we're either a filter or a subquery, we have to default to
+    // not wrapping anything...
+    if (isFilter() || isSubQuery) {
+      return null;
+    }
+
+    // At this point we now are a (regular) query and can wrap global `fq` filters...
+    try {
+      // Start by assuming we wrap all global filters,
+      // then adjust our list based on include/exclude tag params
+      List<Query> globalFQs = QueryUtils.parseFilterQueries(req);
+
+      // Adjust our globalFQs based on any include/exclude we may have
+      if (!includedGlobalFQTags.isEmpty()) {
+        // NOTE: Even if no FQs match the specified tag(s) the fact that tags were specified
+        // means we should replace globalFQs (even with a possibly empty list)
+        globalFQs = new ArrayList<>(QueryUtils.getTaggedQueries(req, includedGlobalFQTags));
+      }
+      if (null != excludedGlobalFQTags) {
+        globalFQs.removeAll(QueryUtils.getTaggedQueries(req, excludedGlobalFQTags));
+      }
+
+      return req.getSearcher().getProcessedFilter(globalFQs).filter;
+
+    } catch (IOException e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+    }
+  }
+
+  /**
+   * @return set (possibly empty) of tags specified in the given local param
+   * @see StrUtils#splitSmart
+   * @see QueryUtils#getTaggedQueries
+   * @see #localParams
+   */
+  private List<String> getLocalParamTags(final String param) {
+    final String[] strVals = localParams.getParams(param);
+    if (null == strVals) {
+      return Collections.emptyList();
+    }
+    final List<String> tags = new ArrayList<>(strVals.length * 2);
+    for (String val : strVals) {
+      // This ensures parity w/how QParser constructor builds tagMap,
+      // and that empty strings will make it into our List (for "include nothing")
+      if (0 < val.indexOf(',')) {
+        tags.addAll(StrUtils.splitSmart(val, ','));
+      } else {
+        tags.add(val);
+      }
     }
-    return null;
+    return tags;
   }
 }
diff --git a/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java b/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java
index 0a03521c87b..8eb1c3da71f 100644
--- a/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java
+++ b/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java
@@ -1350,9 +1350,65 @@ public class QueryEqualityTest extends SolrTestCaseJ4 {
     assertU(adoc(doc));
     assertU(commit());
 
-    try {
-      assertQueryEquals(
-          "knn", "{!knn f=vector}[1.0,2.0,3.0,4.0]", "{!knn f=vector v=[1.0,2.0,3.0,4.0]}");
+    final String qvec = "[1.0,2.0,3.0,4.0]";
+
+    try (SolrQueryRequest req0 = req()) {
+
+      // no filters
+      final Query fqNull =
+          assertQueryEqualsAndReturn(
+              "knn",
+              req0,
+              "{!knn f=vector}" + qvec,
+              "{!knn f=vector preFilter=''}" + qvec,
+              "{!knn f=vector v=" + qvec + "}");
+
+      try (SolrQueryRequest req1 = req("fq", "{!tag=t1}id:1", "xxx", "id:1")) {
+        // either global fq, or (same) preFilter as localparam
+        final Query fqOne =
+            assertQueryEqualsAndReturn(
+                "knn",
+                req1,
+                "{!knn f=vector}" + qvec,
+                "{!knn f=vector includeTags=t1}" + qvec,
+                "{!knn f=vector preFilter='id:1'}" + qvec,
+                "{!knn f=vector preFilter=$xxx}" + qvec,
+                "{!knn f=vector v=" + qvec + "}");
+        QueryUtils.checkUnequal(fqNull, fqOne);
+
+        try (SolrQueryRequest req2 = req("fq", "{!tag=t2}id:2", "xxx", "id:1", "yyy", "")) {
+          // override global fq with local param to use different preFilter
+          final Query fqOneOverride =
+              assertQueryEqualsAndReturn(
+                  "knn",
+                  req2,
+                  "{!knn f=vector preFilter='id:1'}" + qvec,
+                  "{!knn f=vector preFilter=$xxx}" + qvec);
+          QueryUtils.checkEqual(fqOne, fqOneOverride);
+
+          // override global fq with local param to use no preFilters
+          final Query fqNullOverride =
+              assertQueryEqualsAndReturn(
+                  "knn",
+                  req2,
+                  "{!knn f=vector preFilter=''}" + qvec,
+                  "{!knn f=vector excludeTags=t2}" + qvec,
+                  "{!knn f=vector preFilter=$yyy}" + qvec);
+          QueryUtils.checkEqual(fqNull, fqNullOverride);
+        }
+      }
+
+      try (SolrQueryRequest reqPostFilter = req("fq", "{!tag=post frange cache=false l=0}9.9")) {
+        // global post-filter fq should always be ignored
+        final Query fqPostFilter =
+            assertQueryEqualsAndReturn(
+                "knn",
+                reqPostFilter,
+                "{!knn f=vector}" + qvec,
+                "{!knn f=vector includeTags=post}" + qvec);
+        QueryUtils.checkEqual(fqNull, fqPostFilter);
+      }
+
     } finally {
       delQ("id:0");
       assertU(commit());
@@ -1364,12 +1420,12 @@ public class QueryEqualityTest extends SolrTestCaseJ4 {
    * for coverage sanity checking
    *
    * @see #testParserCoverage
-   * @see #assertQueryEquals
+   * @see #assertQueryEqualsAndReturn
    */
   protected void assertQueryEquals(final String defType, final String... inputs) throws Exception {
     SolrQueryRequest req = req(new String[] {"df", "text"});
     try {
-      assertQueryEquals(defType, req, inputs);
+      assertQueryEqualsAndReturn(defType, req, inputs);
     } finally {
       req.close();
     }
@@ -1379,13 +1435,34 @@ public class QueryEqualityTest extends SolrTestCaseJ4 {
    * NOTE: defType is not only used to pick the parser, but, if non-null it is also to record the
    * parser being tested for coverage sanity checking
    *
+   * @see #testParserCoverage
+   * @see #assertQueryEqualsAndReturn
+   */
+  protected void assertQueryEquals(
+      final String defType, final SolrQueryRequest req, final String... inputs) throws Exception {
+    assertQueryEqualsAndReturn(defType, req, inputs);
+  }
+
+  /**
+   * Parses a set of input strings in the context of a request, making various assertions about the
+   * resulting Query objects, including that they must all be equals.
+   *
+   * <p>Returns one of the (all equal) Query objects so it may be used in other comparisons with
+   * other Query objects, possibly parsed in the context of different requests.
+   *
+   * <p>NOTE: defType is not only used to pick the parser, but, if non-null it is also to record the
+   * parser being tested for coverage sanity checking.
+   *
    * @see QueryUtils#check
    * @see QueryUtils#checkEqual
    * @see #testParserCoverage
    */
-  protected void assertQueryEquals(
+  protected Query assertQueryEqualsAndReturn(
       final String defType, final SolrQueryRequest req, final String... inputs) throws Exception {
 
+    assertTrue(
+        "At least one input string for parsing must be passed to this method", 0 < inputs.length);
+
     if (null != defType) qParsersTested.add(defType);
 
     final Query[] queries = new Query[inputs.length];
@@ -1409,6 +1486,7 @@ public class QueryEqualityTest extends SolrTestCaseJ4 {
         QueryUtils.checkEqual(query1, query2);
       }
     }
+    return queries[0];
   }
 
   /**
diff --git a/solr/core/src/test/org/apache/solr/search/neural/KnnQParserTest.java b/solr/core/src/test/org/apache/solr/search/neural/KnnQParserTest.java
index 5cf2fd41f97..ccd0e3ecc44 100644
--- a/solr/core/src/test/org/apache/solr/search/neural/KnnQParserTest.java
+++ b/solr/core/src/test/org/apache/solr/search/neural/KnnQParserTest.java
@@ -26,6 +26,8 @@ import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.request.SolrQueryRequest;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -453,6 +455,8 @@ public class KnnQParserTest extends SolrTestCaseJ4 {
   @Test
   public void knnQueryUsedInFilters_shouldFilterResultsBeforeTheQueryExecution() {
     String vectorToSearch = "[1.0, 2.0, 3.0, 4.0]";
+
+    // topK=4 -> 1,4,2,10
     assertQ(
         req(
             CommonParams.Q,
@@ -460,7 +464,7 @@ public class KnnQParserTest extends SolrTestCaseJ4 {
             "fq",
             "{!knn f=vector topK=4}" + vectorToSearch,
             "fq",
-            "id:(4 20)",
+            "id:(4 20 9)",
             "fl",
             "id"),
         "//result[@numFound='1']",
@@ -468,37 +472,426 @@ public class KnnQParserTest extends SolrTestCaseJ4 {
   }
 
   @Test
-  public void knnQueryWithFilterQuery_shouldPerformKnnSearchInPreFilteredResults() {
+  public void knnQueryUsedInFiltersWithPreFilter_shouldFilterResultsBeforeTheQueryExecution() {
     String vectorToSearch = "[1.0, 2.0, 3.0, 4.0]";
 
+    // topK=4 w/localparam preFilter -> 1,4,7,9
     assertQ(
         req(
             CommonParams.Q,
-            "{!knn f=vector topK=10}" + vectorToSearch,
+            "id:(3 4 9 2)",
+            "fq",
+            "{!knn f=vector topK=4 preFilter='id:(1 4 7 8 9)'}" + vectorToSearch,
             "fq",
-            "id:(1 2 7 20)",
+            "id:(4 20 9)",
             "fl",
             "id"),
+        "//result[@numFound='2']",
+        "//result/doc[1]/str[@name='id'][.='4']",
+        "//result/doc[2]/str[@name='id'][.='9']");
+  }
+
+  @Test
+  public void knnQueryUsedInFilters_rejectIncludeExclude() {
+    String vectorToSearch = "[1.0, 2.0, 3.0, 4.0]";
+
+    for (String fq :
+        Arrays.asList(
+            "{!knn f=vector topK=5 includeTags=xxx}" + vectorToSearch,
+            "{!knn f=vector topK=5 excludeTags=xxx}" + vectorToSearch)) {
+      assertQEx(
+          "fq={!knn...} incompatible with include/exclude localparams",
+          "used as a filter does not support",
+          req("q", "*:*", "fq", fq),
+          SolrException.ErrorCode.BAD_REQUEST);
+    }
+  }
+
+  @Test
+  public void knnQueryAsSubQuery() {
+    final SolrParams common = params("fl", "id", "vec", "[1.0, 2.0, 3.0, 4.0]");
+    final String filt = "id:(2 4 7 9 8 20 3)";
+
+    // When knn parser is a subquery, it should not pre-filter on any global fq params
+    // topK -> 1,4,2,10,3 -> fq -> 4,2,3
+    assertQ(
+        req(common, "fq", filt, "q", "*:* AND {!knn f=vector topK=5 v=$vec}"),
         "//result[@numFound='3']",
-        "//result/doc[1]/str[@name='id'][.='1']",
+        "//result/doc[1]/str[@name='id'][.='4']",
         "//result/doc[2]/str[@name='id'][.='2']",
-        "//result/doc[3]/str[@name='id'][.='7']");
+        "//result/doc[3]/str[@name='id'][.='3']");
+    // topK -> 1,4,2,10,3 + '8' -> fq -> 4,2,3,8
+    assertQ(
+        req(common, "fq", filt, "q", "id:8^=0.01 OR {!knn f=vector topK=5 v=$vec}"),
+        "//result[@numFound='4']",
+        "//result/doc[1]/str[@name='id'][.='4']",
+        "//result/doc[2]/str[@name='id'][.='2']",
+        "//result/doc[3]/str[@name='id'][.='3']",
+        "//result/doc[4]/str[@name='id'][.='8']");
+  }
+
+  @Test
+  public void knnQueryAsSubQuery_withPreFilter() {
+    final SolrParams common = params("fl", "id", "vec", "[1.0, 2.0, 3.0, 4.0]");
+    final String filt = "id:(2 4 7 9 8 20 3)";
 
+    // knn subquery should still accept `preFilter` local param
+    // filt -> topK -> 4,2,3,7,9
+    assertQ(
+        req(common, "q", "*:* AND {!knn f=vector topK=5 preFilter='" + filt + "' v=$vec}"),
+        "//result[@numFound='5']",
+        "//result/doc[1]/str[@name='id'][.='4']",
+        "//result/doc[2]/str[@name='id'][.='2']",
+        "//result/doc[3]/str[@name='id'][.='3']",
+        "//result/doc[4]/str[@name='id'][.='7']",
+        "//result/doc[5]/str[@name='id'][.='9']");
+
+    // it should not pre-filter on any global fq params
+    // filt -> topK -> 4,2,3,7,9 -> fq -> 3,9
     assertQ(
         req(
-            CommonParams.Q,
-            "{!knn f=vector topK=4}" + vectorToSearch,
+            common,
             "fq",
-            "id:(3 4 9 2)",
-            "fl",
-            "id"),
+            "id:(1 9 20 3 5 6 8)",
+            "q",
+            "*:* AND {!knn f=vector topK=5 preFilter='" + filt + "' v=$vec}"),
+        "//result[@numFound='2']",
+        "//result/doc[1]/str[@name='id'][.='3']",
+        "//result/doc[2]/str[@name='id'][.='9']");
+    // filt -> topK -> 4,2,3,7,9 + '8' -> fq -> 8,3,9
+    assertQ(
+        req(
+            common,
+            "fq",
+            "id:(1 9 20 3 5 6 8)",
+            "q",
+            "id:8^=100 OR {!knn f=vector topK=5 preFilter='" + filt + "' v=$vec}"),
+        "//result[@numFound='3']",
+        "//result/doc[1]/str[@name='id'][.='8']",
+        "//result/doc[2]/str[@name='id'][.='3']",
+        "//result/doc[3]/str[@name='id'][.='9']");
+  }
+
+  @Test
+  public void knnQueryAsSubQuery_rejectIncludeExclude() {
+    final SolrParams common = params("fl", "id", "vec", "[1.0, 2.0, 3.0, 4.0]");
+
+    for (String knn :
+        Arrays.asList(
+            "{!knn f=vector topK=5 includeTags=xxx v=$vec}",
+            "{!knn f=vector topK=5 excludeTags=xxx v=$vec}")) {
+      assertQEx(
+          "knn as subquery incompatible with include/exclude localparams",
+          "used as a sub-query does not support",
+          req(common, "q", "*:* OR " + knn),
+          SolrException.ErrorCode.BAD_REQUEST);
+    }
+  }
+
+  @Test
+  public void knnQueryWithFilterQuery_singlePreFilterEquivilence() {
+    final String vectorToSearch = "[1.0, 2.0, 3.0, 4.0]";
+    final SolrParams common = params("fl", "id");
+
+    // these requests should be equivalent
+    final String filt = "id:(1 2 7 20)";
+    for (SolrQueryRequest req :
+        Arrays.asList(
+            req(common, "q", "{!knn f=vector topK=10}" + vectorToSearch, "fq", filt),
+            req(common, "q", "{!knn f=vector preFilter=\"" + filt + "\" topK=10}" + vectorToSearch),
+            req(
+                common,
+                "q",
+                "{!knn f=vector preFilter=$my_filt topK=10}" + vectorToSearch,
+                "my_filt",
+                filt))) {
+      assertQ(
+          req,
+          "//result[@numFound='3']",
+          "//result/doc[1]/str[@name='id'][.='1']",
+          "//result/doc[2]/str[@name='id'][.='2']",
+          "//result/doc[3]/str[@name='id'][.='7']");
+    }
+  }
+
+  @Test
+  public void knnQueryWithFilterQuery_multiPreFilterEquivilence() {
+    final String vectorToSearch = "[1.0, 2.0, 3.0, 4.0]";
+    final SolrParams common = params("fl", "id");
+
+    // these requests should be equivalent
+    final String fx = "id:(3 4 9 2 1 )"; // 1 & 10 dropped from intersection
+    final String fy = "id:(3 4 9 2 10)";
+    for (SolrQueryRequest req :
+        Arrays.asList(
+            req(common, "q", "{!knn f=vector topK=4}" + vectorToSearch, "fq", fx, "fq", fy),
+            req(
+                common,
+                "q",
+                "{!knn f=vector preFilter=\""
+                    + fx
+                    + "\" preFilter=\""
+                    + fy
+                    + "\" topK=4}"
+                    + vectorToSearch),
+            req(
+                common,
+                "q",
+                "{!knn f=vector preFilter=$fx preFilter=$fy topK=4}" + vectorToSearch,
+                "fx",
+                fx,
+                "fy",
+                fy),
+            req(
+                common,
+                "q",
+                "{!knn f=vector preFilter=$multi_filt topK=4}" + vectorToSearch,
+                "multi_filt",
+                fx,
+                "multi_filt",
+                fy))) {
+      assertQ(
+          req,
+          "//result[@numFound='4']",
+          "//result/doc[1]/str[@name='id'][.='4']",
+          "//result/doc[2]/str[@name='id'][.='2']",
+          "//result/doc[3]/str[@name='id'][.='3']",
+          "//result/doc[4]/str[@name='id'][.='9']");
+    }
+  }
+
+  @Test
+  public void knnQueryWithPreFilter_rejectIncludeExclude() {
+    final String vectorToSearch = "[1.0, 2.0, 3.0, 4.0]";
+
+    assertQEx(
+        "knn preFilter localparm incompatible with include/exclude localparams",
+        "does not support combining preFilter localparam with either",
+        // shouldn't matter if global fq w/tag even exists, usage is an error
+        req("q", "{!knn f=vector preFilter='id:1' includeTags=xxx}" + vectorToSearch),
+        SolrException.ErrorCode.BAD_REQUEST);
+    assertQEx(
+        "knn preFilter localparm incompatible with include/exclude localparams",
+        "does not support combining preFilter localparam with either",
+        // shouldn't matter if global fq w/tag even exists, usage is an error
+        req("q", "{!knn f=vector preFilter='id:1' excludeTags=xxx}" + vectorToSearch),
+        SolrException.ErrorCode.BAD_REQUEST);
+  }
+
+  @Test
+  public void knnQueryWithFilterQuery_preFilterLocalParamOverridesGlobalFilters() {
+    final String vectorToSearch = "[1.0, 2.0, 3.0, 4.0]";
+
+    // trivial case: empty preFilter localparam means no pre-filtering
+    assertQ(
+        req(
+            "q", "{!knn f=vector preFilter='' topK=5}" + vectorToSearch,
+            "fq", "-id:4",
+            "fl", "id"),
         "//result[@numFound='4']",
-        "//result/doc[1]/str[@name='id'][.='4']",
+        "//result/doc[1]/str[@name='id'][.='1']",
         "//result/doc[2]/str[@name='id'][.='2']",
-        "//result/doc[3]/str[@name='id'][.='3']",
+        "//result/doc[3]/str[@name='id'][.='10']",
+        "//result/doc[4]/str[@name='id'][.='3']");
+
+    // localparam prefiltering, global fqs applied independently
+    assertQ(
+        req(
+            "q", "{!knn f=vector preFilter='id:(3 4 9 2 7 8)' topK=5}" + vectorToSearch,
+            "fq", "-id:4",
+            "fl", "id"),
+        "//result[@numFound='4']",
+        "//result/doc[1]/str[@name='id'][.='2']",
+        "//result/doc[2]/str[@name='id'][.='3']",
+        "//result/doc[3]/str[@name='id'][.='7']",
         "//result/doc[4]/str[@name='id'][.='9']");
   }
 
+  @Test
+  public void knnQueryWithFilterQuery_localParamIncludeExcludeTags() {
+    final String vectorToSearch = "[1.0, 2.0, 3.0, 4.0]";
+    final SolrParams common =
+        params(
+            "fl", "id",
+            "fq", "{!tag=xx,aa}id:(5 6 7 8 9 10)",
+            "fq", "{!tag=yy,aa}id:(1 2 3 4 5 6 7)");
+
+    // These req's are equivalent: pre-filter everything
+    // So only 7,6,5 are viable for topK=5
+    for (SolrQueryRequest req :
+        Arrays.asList(
+            // default behavior is all fq's pre-filter,
+            req(common, "q", "{!knn f=vector topK=5}" + vectorToSearch),
+            // diff ways of explicitly requesting both fq params
+            req(common, "q", "{!knn f=vector includeTags=aa topK=5}" + vectorToSearch),
+            req(
+                common,
+                "q",
+                "{!knn f=vector includeTags=aa excludeTags='' topK=5}" + vectorToSearch),
+            req(
+                common,
+                "q",
+                "{!knn f=vector includeTags=aa excludeTags=bogus topK=5}" + vectorToSearch),
+            req(
+                common,
+                "q",
+                "{!knn f=vector includeTags=xx includeTags=yy topK=5}" + vectorToSearch),
+            req(common, "q", "{!knn f=vector includeTags=xx,yy,bogus topK=5}" + vectorToSearch))) {
+      assertQ(
+          req,
+          "//result[@numFound='3']",
+          "//result/doc[1]/str[@name='id'][.='7']",
+          "//result/doc[2]/str[@name='id'][.='5']",
+          "//result/doc[3]/str[@name='id'][.='6']");
+    }
+  }
+
+  @Test
+  public void knnQueryWithFilterQuery_localParamsDisablesAllPreFiltering() {
+    final String vectorToSearch = "[1.0, 2.0, 3.0, 4.0]";
+    final SolrParams common =
+        params(
+            "fl", "id",
+            "fq", "{!tag=xx,aa}id:(5 6 7 8 9 10)",
+            "fq", "{!tag=yy,aa}id:(1 2 3 4 5 6 7)");
+
+    // These req's are equivalent: pre-filter nothing
+    // So 1,4,2,10,3,7 are the topK=6
+    // Only 7 matches both of the the regular fq params
+    for (SolrQueryRequest req :
+        Arrays.asList(
+            // explicit local empty preFilter
+            req(common, "q", "{!knn f=vector preFilter='' topK=6}" + vectorToSearch),
+            // diff ways of explicitly including none of the global fq params
+            req(common, "q", "{!knn f=vector includeTags='' topK=6}" + vectorToSearch),
+            req(common, "q", "{!knn f=vector includeTags=bogus topK=6}" + vectorToSearch),
+            // diff ways of explicitly excluding all of the global fq params
+            req(common, "q", "{!knn f=vector excludeTags=aa topK=6}" + vectorToSearch),
+            req(
+                common,
+                "q",
+                "{!knn f=vector includeTags=aa excludeTags=aa topK=6}" + vectorToSearch),
+            req(
+                common,
+                "q",
+                "{!knn f=vector includeTags=aa excludeTags=xx,yy topK=6}" + vectorToSearch),
+            req(
+                common,
+                "q",
+                "{!knn f=vector includeTags=xx,yy excludeTags=aa topK=6}" + vectorToSearch),
+            req(common, "q", "{!knn f=vector excludeTags=xx,yy topK=6}" + vectorToSearch),
+            req(common, "q", "{!knn f=vector excludeTags=aa topK=6}" + vectorToSearch),
+            req(
+                common,
+                "q",
+                "{!knn f=vector excludeTags=xx excludeTags=yy topK=6}" + vectorToSearch),
+            req(
+                common,
+                "q",
+                "{!knn f=vector excludeTags=xx excludeTags=yy,bogus topK=6}" + vectorToSearch),
+            req(common, "q", "{!knn f=vector excludeTags=xx,yy,bogus topK=6}" + vectorToSearch))) {
+      assertQ(req, "//result[@numFound='1']", "//result/doc[1]/str[@name='id'][.='7']");
+    }
+  }
+
+  @Test
+  public void knnQueryWithFilterQuery_localParamCombinedIncludeExcludeTags() {
+    final String vectorToSearch = "[1.0, 2.0, 3.0, 4.0]";
+    final SolrParams common =
+        params(
+            "fl", "id",
+            "fq", "{!tag=xx,aa}id:(5 6 7 8 9 10)",
+            "fq", "{!tag=yy,aa}id:(1 2 3 4 5 6 7)");
+
+    // These req's are equivalent: prefilter only the 'yy' fq
+    // So 1,4,2,3,7 are in the topK=5.
+    // Only 7 matches the regular 'xx' fq param
+    for (SolrQueryRequest req :
+        Arrays.asList(
+            // diff ways of only using the 'yy' filter
+            req(common, "q", "{!knn f=vector includeTags=yy,bogus topK=5}" + vectorToSearch),
+            req(
+                common,
+                "q",
+                "{!knn f=vector includeTags=yy excludeTags='' topK=5}" + vectorToSearch),
+            req(common, "q", "{!knn f=vector excludeTags=xx,bogus topK=5}" + vectorToSearch),
+            req(
+                common,
+                "q",
+                "{!knn f=vector includeTags=yy excludeTags=xx topK=5}" + vectorToSearch),
+            req(
+                common,
+                "q",
+                "{!knn f=vector includeTags=aa excludeTags=xx topK=5}" + vectorToSearch))) {
+      assertQ(req, "//result[@numFound='1']", "//result/doc[1]/str[@name='id'][.='7']");
+    }
+  }
+
+  @Test
+  public void knnQueryWithMultiSelectFaceting_excludeTags() {
+    // NOTE: faceting on id is not very realistic,
+    // but it confirms what we care about re:filters w/o needing extra fields.
+    final String facet_xpath = "//lst[@name='facet_fields']/lst[@name='id']/int";
+    final String vectorToSearch = "[1.0, 2.0, 3.0, 4.0]";
+
+    final SolrParams common =
+        params(
+            "fl", "id",
+            "indent", "true",
+            "q", "{!knn f=vector topK=5 excludeTags=facet_click v=$vec}",
+            "vec", vectorToSearch,
+            // mimicing "inStock:true"
+            "fq", "-id:(2 3)",
+            "facet", "true",
+            "facet.mincount", "1",
+            "facet.field", "{!ex=facet_click}id");
+
+    // initial query, with basic pre-filter and facet counts
+    assertQ(
+        req(common),
+        "//result[@numFound='5']",
+        "//result/doc[1]/str[@name='id'][.='1']",
+        "//result/doc[2]/str[@name='id'][.='4']",
+        "//result/doc[3]/str[@name='id'][.='10']",
+        "//result/doc[4]/str[@name='id'][.='7']",
+        "//result/doc[5]/str[@name='id'][.='5']",
+        "*[count(" + facet_xpath + ")=5]",
+        facet_xpath + "[@name='1'][.='1']",
+        facet_xpath + "[@name='4'][.='1']",
+        facet_xpath + "[@name='10'][.='1']",
+        facet_xpath + "[@name='7'][.='1']",
+        facet_xpath + "[@name='5'][.='1']");
+
+    // drill down on a single facet constraint
+    // multi-select means facet counts shouldn't change
+    // (this proves the knn isn't pre-filtering on the 'facet_click' fq)
+    assertQ(
+        req(common, "fq", "{!tag=facet_click}id:(4)"),
+        "//result[@numFound='1']",
+        "//result/doc[1]/str[@name='id'][.='4']",
+        "*[count(" + facet_xpath + ")=5]",
+        facet_xpath + "[@name='1'][.='1']",
+        facet_xpath + "[@name='4'][.='1']",
+        facet_xpath + "[@name='10'][.='1']",
+        facet_xpath + "[@name='7'][.='1']",
+        facet_xpath + "[@name='5'][.='1']");
+
+    // drill down on an additional facet constraint
+    // multi-select means facet counts shouldn't change
+    // (this proves the knn isn't pre-filtering on the 'facet_click' fq)
+    assertQ(
+        req(common, "fq", "{!tag=facet_click}id:(4 5)"),
+        "//result[@numFound='2']",
+        "//result/doc[1]/str[@name='id'][.='4']",
+        "//result/doc[2]/str[@name='id'][.='5']",
+        "*[count(" + facet_xpath + ")=5]",
+        facet_xpath + "[@name='1'][.='1']",
+        facet_xpath + "[@name='4'][.='1']",
+        facet_xpath + "[@name='10'][.='1']",
+        facet_xpath + "[@name='7'][.='1']",
+        facet_xpath + "[@name='5'][.='1']");
+  }
+
   @Test
   public void knnQueryWithCostlyFq_shouldPerformKnnSearchWithPostFilter() {
     String vectorToSearch = "[1.0, 2.0, 3.0, 4.0]";
diff --git a/solr/solr-ref-guide/modules/query-guide/pages/dense-vector-search.adoc b/solr/solr-ref-guide/modules/query-guide/pages/dense-vector-search.adoc
index 24d7859bb39..ac96ef827bf 100644
--- a/solr/solr-ref-guide/modules/query-guide/pages/dense-vector-search.adoc
+++ b/solr/solr-ref-guide/modules/query-guide/pages/dense-vector-search.adoc
@@ -240,7 +240,7 @@ client.add(Arrays.asList(d1, d2));
 This is the Apache Solr query approach designed to support dense vector search:
 
 === knn Query Parser
-The `knn` k-nearest neighbors query parser allows to find the k-nearest documents to the target vector according to indexed dense vectors in the given field.
+The `knn` k-nearest neighbors query parser allows to find the k-nearest documents to the target vector according to indexed dense vectors in the given field.  The set of documents can be Pre-Filtered to reduce the number of vector distance calculations that must be computed, and ensure the best `topK` are returned.
 
 The score for a retrieved document is the approximate distance to the target vector(defined by the similarityFunction configured at indexing time).
 
@@ -264,45 +264,113 @@ The `DenseVectorField` to search in.
 +
 How many k-nearest results to return.
 
-Here's how to run a KNN search:
+`preFilter`::
++
+[%autowidth,frame=none]
+|===
+|Optional |Default: Depends on usage, see below.
+|===
++
+Specifies an explicit list of Pre-Filter query strings to use.
 
-[source,text]
-&q={!knn f=vector topK=10}[1.0, 2.0, 3.0, 4.0]
+`includeTags`::
++
+[%autowidth,frame=none]
+|===
+|Optional |Default: none
+|===
++
+Indicates that only `fq` filters with the specified `tag` should be considered for implicit Pre-Filtering.  Must not be combined with `preFilter`.
 
-The search results retrieved are the k-nearest to the vector in input `[1.0, 2.0, 3.0, 4.0]`, ranked by the similarityFunction configured at indexing time.
 
-==== Usage with Filter Queries
-The `knn` query parser can be used in filter queries:
+`excludeTags`::
++
+[%autowidth,frame=none]
+|===
+|Optional |Default: none
+|===
++
+Indicates that `fq` filters with the specified `tag` should be excluded from consideration for implicit Pre-Filtering.  Must not be combined with `preFilter`.
+
+
+Here's how to run a simple KNN search:
+
 [source,text]
-&q=id:(1 2 3)&fq={!knn f=vector topK=10}[1.0, 2.0, 3.0, 4.0]
+?q={!knn f=vector topK=10}[1.0, 2.0, 3.0, 4.0]
+
+The search results retrieved are the k=10 nearest documents to the vector in input `[1.0, 2.0, 3.0, 4.0]`, ranked by the `similarityFunction` configured at indexing time.
+
+
+==== Explicit KNN Pre-Filtering
+
+The `knn` query parser's `preFilter` parameter can be specified to reduce the number of candidate documents evaluated for the k-nearest distance calculation:
 
-The `knn` query parser can be used with filter queries:
 [source,text]
-&q={!knn f=vector topK=10}[1.0, 2.0, 3.0, 4.0]&fq=id:(1 2 3)
+?q={!knn f=vector topK=10 preFilter=inStock:true}[1.0, 2.0, 3.0, 4.0]
 
-[IMPORTANT]
-====
-Filter queries are executed as pre-filters: the main query refines the sub-set of search results derived from the application of all the filter queries combined as 'MUST' clauses(boolean AND).
+In the above example, only documents matching the Pre-Filter `inStock:true` will be candidates for consideration when evaluating the k-nearest search against the specified vector.
+
+The `preFilter` parameter may be blank (ex: `preFilter=""`) to indicate that no Pre-Filtering should be performed; or it may be multi-valued -- either through repetition, or via duplicated xref:local-params.adoc#parameter-dereferencing[Parameter References].
+
+These two examples are equivalent:
+
+[source,text]
+?q={!knn f=vector topK=10 preFilter=category:AAA preFilter=inStock:true}[1.0, 2.0, 3.0, 4.0]
 
-This means that in
 [source,text]
-&q=id:(1 2 3)&fq={!knn f=vector topK=10}[1.0, 2.0, 3.0, 4.0]
+----
+?q={!knn f=vector topK=10 preFilter=$knnPreFilter}[1.0, 2.0, 3.0, 4.0]
+&knnPreFilter=category:AAA
+&knnPreFilter=inStock:true
+----
 
-The results are prefiltered by the topK knn retrieval and then only the documents from this subset, matching the query 'q=id:(1 2 3)' are returned.
+==== Implicit KNN Pre-Filtering
+
+While the `preFilter` parameter may be explicitly specified on *_any_* usage of the `knn` query parser, the default Pre-Filtering behavior (when no `preFilter` parameter is specified) will vary based on how the `knn` query parser is used:
+
+* When used as the main `q` param: `fq` filters in the request (that are not xref:common-query-parameters.adoc#cache-local-parameter[Solr Post Filters]) will be combined to form an implicit KNN Pre-Filter.
+** This default behavior optimizes the number of vector distance calculations considered, eliminating documents that would eventually be excluded by an `fq` filter anyway.
+** `includeTags` and `excludeTags` may be used to limit the set of `fq` filters used in the Pre-Filter.
+* When used as an `fq` param, or as a subquery clause in a larger query: No implicit Pre-Filter is used.
+** `includeTags` and `excludeTags` must not be used in these situations.
+
+
+The example request below shows two usages of the `knn` query parser that will get _no_ implicit Pre-Filtering from any of the `fq` parameters, because neither usage is as the main `q` param:
 
-In
 [source,text]
-&q={!knn f=vector topK=10}[1.0, 2.0, 3.0, 4.0]&fq=id:(1 2 3)
+----
+?q=(color_str:red OR {!knn f=color_vector topK=10 v="[1.0, 2.0, 3.0, 4.0]"})
+&fq={!knn f=title_vector topK=10}[9.0, 8.0, 7.0, 6.0]
+&fq=inStock:true
+----
 
-The results are prefiltered by the fq=id:(1 2 3) and then only the documents from this subset are considered as candidates for the topK knn retrieval.
 
-If you want to run some of the filter queries as post-filters you can follow the standard approach for post-filtering in Apache Solr, using the cache and cost local parameters.
+However, the next example shows a basic request where all `fq` parameters will be used as implicit Pre-Filters on the main `knn` query:
 
-e.g.
+[source,text]
+----
+?q={!knn f=vector topK=10}[1.0, 2.0, 3.0, 4.0]
+&fq=category:AAA
+&fq=inStock:true
+----
+
+If we modify the above request to add tags to the `fq` parameters, we can specify an `includeTags` option on the `knn` parser to limit which `fq` filters are used for Pre-Filtering:
 
 [source,text]
-&q={!knn f=vector topK=10}[1.0, 2.0, 3.0, 4.0]&fq={!frange cache=false l=0.99}$q
-====
+----
+?q={!knn f=vector topK=10 includeTags=for_knn}[1.0, 2.0, 3.0, 4.0]
+&fq=category:AAA
+&fq={!tag=for_knn}inStock:true
+----
+
+In this example, only the `inStock:true` filter will be used for KNN Pre-Filtering to find the the `topK=10` documents, and the `category:AAA` filter will be applied independently; possibly resulting in less then 10 total matches.
+
+
+Some use cases where `includeTags` and/or `excludeTags` may be more useful then an explicit `preFilter` parameters:
+
+* You have some `fq` parameters that are xref:configuration-guide:requesthandlers-searchcomponents.adoc#paramsets-and-useparams[re-used on many requests] (even when you don't use the `knn` parser) that you wish to be used as KNN Pre-Filters when you _do_ use the `knn` query parser.
+* You typically want all `fq` params to be used as KNN Pre-Filters, but when users "drill down" on Facets, you want the `fq` parameters you add to be excluded from the KNN Pre-Filtering so that the result set gets smaller; instead of just computing a new `topK` set.
+
 
 
 ==== Usage as Re-Ranking Query