You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2022/02/07 23:54:14 UTC

[GitHub] [lucene] jtibshirani opened a new pull request #656: LUCENE-10382: Support filtering in KnnVectorQuery

jtibshirani opened a new pull request #656:
URL: https://github.com/apache/lucene/pull/656


   This PR adds support for a query filter in KnnVectorQuery. First, we gather the
   query results for each leaf as a bit set. Then the HNSW search skips over the
   non-matching documents (using the same approach as for live docs). To prevent
   HNSW search from visiting too many documents when the filter is very selective,
   we short-circuit if HNSW has already visited more than the number of documents
   that match the filter, and execute an exact search instead. This bounds the
   number of visited documents at roughly 2x the cost of just running the exact
   filter, while in most cases HNSW completes successfully and does a lot better.
   
   Co-authored-by: Joel Bernstein <jb...@apache.org>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jtibshirani commented on a change in pull request #656: LUCENE-10382: Support filtering in KnnVectorQuery

Posted by GitBox <gi...@apache.org>.
jtibshirani commented on a change in pull request #656:
URL: https://github.com/apache/lucene/pull/656#discussion_r803937272



##########
File path: lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java
##########
@@ -96,43 +107,98 @@ public Query rewrite(IndexReader reader) throws IOException {
     return createRewrittenQuery(reader, topK);
   }
 
-  private TopDocs searchLeaf(LeafReaderContext ctx, int kPerLeaf, Bits bitsFilter)
+  private TopDocs searchLeaf(LeafReaderContext ctx, int kPerLeaf, BitSetCollector filterCollector)
       throws IOException {
-    // If the filter is non-null, then it already handles live docs
-    if (bitsFilter == null) {
-      bitsFilter = ctx.reader().getLiveDocs();
+
+    if (filterCollector == null) {
+      Bits acceptDocs = ctx.reader().getLiveDocs();
+      return ctx.reader()
+          .searchNearestVectors(field, target, kPerLeaf, acceptDocs, Integer.MAX_VALUE);
+    } else {
+      BitSetIterator filterIterator = filterCollector.getIterator(ctx.ord);
+      if (filterIterator == null || filterIterator.cost() == 0) {

Review comment:
       I can add a comment explaining how I'm using the `BitSetIterator` here to capture both the bitset and the (exact) cardinality.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jtibshirani commented on a change in pull request #656: LUCENE-10382: Support filtering in KnnVectorQuery

Posted by GitBox <gi...@apache.org>.
jtibshirani commented on a change in pull request #656:
URL: https://github.com/apache/lucene/pull/656#discussion_r802075985



##########
File path: lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91HnswVectorsReader.java
##########
@@ -253,6 +277,36 @@ public TopDocs search(String field, float[] target, int k, Bits acceptDocs) thro
         scoreDocs);
   }
 
+  private TopDocs exactSearch(
+      float[] target,
+      int k,
+      VectorSimilarityFunction similarityFunction,
+      VectorValues vectorValues,
+      DocIdSetIterator acceptIterator)
+      throws IOException {
+    HitQueue topK = new HitQueue(k, false);
+    int numVisited = 0;
+
+    int doc;
+    while ((doc = acceptIterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {

Review comment:
       Good point, I'll rework this.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jtibshirani commented on pull request #656: LUCENE-10382: Support filtering in KnnVectorQuery

Posted by GitBox <gi...@apache.org>.
jtibshirani commented on pull request #656:
URL: https://github.com/apache/lucene/pull/656#issuecomment-1039714498


   @msokolov @jpountz @mayya-sharipova this is ready for another look. Notable changes:
   * When computing the filter results, only include documents that actually contain a vector. This gives an accurate estimate of the filter selectivity. To support this I introduced `KnnVectorFieldExistsQuery`, which seemed useful in its own right.
   * I stopped using `CollectionTerminationException` to indicate that the search hit the visited limit. Instead, we pass the information in `TopDocs` through `TotalHits`. The value is always the number of visited docs, but the relation is `GREATER_THAN_OR_EQUAL_TO` iff the search stopped early. This is kind of arbitrary but felt natural -- I'm very open to suggestions here! It's a fairly low-level API and it's marked experimental, so there is also room to refine it later. This update does not change the output of `KnnVectorQuery`.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jtibshirani commented on pull request #656: LUCENE-10382: Support filtering in KnnVectorQuery

Posted by GitBox <gi...@apache.org>.
jtibshirani commented on pull request #656:
URL: https://github.com/apache/lucene/pull/656#issuecomment-1033082962


   Thanks for reviewing. I'll work on another iteration and ping you when it's out of "draft" status. One clarification first...
   
   > +1 to expose searchExact and allow the Query to call it if it is selective. 
   
   I was thinking we would implement exact search within `KnnVectorQuery` itself, instead of exposing it through `LeafReader`. I think we already have all the pieces we need through the `VectorValues` API. What do you think?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jtibshirani commented on pull request #656: LUCENE-10382: Support filtering in KnnVectorQuery

Posted by GitBox <gi...@apache.org>.
jtibshirani commented on pull request #656:
URL: https://github.com/apache/lucene/pull/656#issuecomment-1040900652


   Thanks for the review! I'll wait for the others to take a look too. I'm working on adding kNN with filtering to luceneutil.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] msokolov commented on pull request #656: LUCENE-10382: Support filtering in KnnVectorQuery

Posted by GitBox <gi...@apache.org>.
msokolov commented on pull request #656:
URL: https://github.com/apache/lucene/pull/656#issuecomment-1033247680


   > I was thinking we would implement exact search within KnnVectorQuery itself, instead of exposing it through LeafReader. I think we already have all the pieces we need through the VectorValues API. What do you think?
   
   Ah I misunderstood. I suppose then in theory the logic can work with any Vector encoding, which would be good, rather than having to reproduce the same brute force approach in each of them


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #656: LUCENE-10382: Support filtering in KnnVectorQuery

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #656:
URL: https://github.com/apache/lucene/pull/656#discussion_r807638868



##########
File path: build.gradle
##########
@@ -183,3 +183,5 @@ apply from: file('gradle/hacks/turbocharge-jvm-opts.gradle')
 apply from: file('gradle/hacks/dummy-outputs.gradle')
 
 apply from: file('gradle/pylucene/pylucene.gradle')
+sourceCompatibility = JavaVersion.VERSION_16

Review comment:
       Are these changes intentional? 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] msokolov commented on a change in pull request #656: LUCENE-10382: Support filtering in KnnVectorQuery

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #656:
URL: https://github.com/apache/lucene/pull/656#discussion_r801992612



##########
File path: lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91HnswVectorsReader.java
##########
@@ -253,6 +277,36 @@ public TopDocs search(String field, float[] target, int k, Bits acceptDocs) thro
         scoreDocs);
   }
 
+  private TopDocs exactSearch(
+      float[] target,
+      int k,
+      VectorSimilarityFunction similarityFunction,
+      VectorValues vectorValues,
+      DocIdSetIterator acceptIterator)
+      throws IOException {
+    HitQueue topK = new HitQueue(k, false);
+    int numVisited = 0;
+
+    int doc;
+    while ((doc = acceptIterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {

Review comment:
       should we call `advance(vectorValues.docID())` here to enable skipping?

##########
File path: lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91HnswVectorsReader.java
##########
@@ -227,16 +231,36 @@ public TopDocs search(String field, float[] target, int k, Bits acceptDocs) thro
 
     // bound k by total number of vectors to prevent oversizing data structures
     k = Math.min(k, fieldEntry.size());
-
     OffHeapVectorValues vectorValues = getOffHeapVectorValues(fieldEntry);
+
+    DocIdSetIterator acceptIterator = null;
+    int visitedLimit = Integer.MAX_VALUE;
+
+    if (acceptDocs instanceof BitSet acceptBitSet) {

Review comment:
       I'm not super-familiar with other algorithms, but it does make sense to me that any approximate algorithm is going to have a "tuning" knob that increases recall in exchange for increased cost. This was the idea behind the now-defunct "fanout" parameter we had in the earlier version of the vector search API. So -- it makes sense to me that we are now bringing back some measure of control over this tuning, albeit in a different form.
   

##########
File path: lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91HnswVectorsReader.java
##########
@@ -227,16 +231,36 @@ public TopDocs search(String field, float[] target, int k, Bits acceptDocs) thro
 
     // bound k by total number of vectors to prevent oversizing data structures
     k = Math.min(k, fieldEntry.size());
-
     OffHeapVectorValues vectorValues = getOffHeapVectorValues(fieldEntry);
+
+    DocIdSetIterator acceptIterator = null;
+    int visitedLimit = Integer.MAX_VALUE;
+
+    if (acceptDocs instanceof BitSet acceptBitSet) {

Review comment:
       @jpountz as always brings up interesting points! - I had no idea we were concerned about the number of subclasses of BitSet, nor was I aware of ExitableDirectoryReader! But I wonder if that should determine the approach here -- should we rely on Bits-based termination, or should we instrument `VectorValues`?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] msokolov commented on a change in pull request #656: LUCENE-10382: Support filtering in KnnVectorQuery

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #656:
URL: https://github.com/apache/lucene/pull/656#discussion_r803668310



##########
File path: lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java
##########
@@ -76,17 +81,23 @@ public KnnVectorQuery(String field, float[] target, int k, Query filter) {
 
   @Override
   public Query rewrite(IndexReader reader) throws IOException {
-    BitSet[] bitSets = null;
+    TopDocs[] perLeafResults = new TopDocs[reader.leaves().size()];
 
+    BitSetCollector filterCollector = null;
     if (filter != null) {
+      filterCollector = new BitSetCollector(reader.leaves().size());
       IndexSearcher indexSearcher = new IndexSearcher(reader);
-      bitSets = new BitSet[reader.leaves().size()];
-      indexSearcher.search(filter, new BitSetCollector(bitSets));
+      indexSearcher.search(filter, filterCollector);

Review comment:
       for another day, but I am realizing that we have no opportunity to make use of per-segment concurrency here, as we ordinarily do in `IndexSearcher.search()`. To do so, we'd need to consider some API change though. Perhaps instead of using `rewrite` for this, we could make use of `Query`'s two-phase iteration mode of operation. Just a thought for later - I'll go open an issue elsewhere.

##########
File path: lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java
##########
@@ -96,43 +107,98 @@ public Query rewrite(IndexReader reader) throws IOException {
     return createRewrittenQuery(reader, topK);
   }
 
-  private TopDocs searchLeaf(LeafReaderContext ctx, int kPerLeaf, Bits bitsFilter)
+  private TopDocs searchLeaf(LeafReaderContext ctx, int kPerLeaf, BitSetCollector filterCollector)
       throws IOException {
-    // If the filter is non-null, then it already handles live docs
-    if (bitsFilter == null) {
-      bitsFilter = ctx.reader().getLiveDocs();
+
+    if (filterCollector == null) {
+      Bits acceptDocs = ctx.reader().getLiveDocs();
+      return ctx.reader()
+          .searchNearestVectors(field, target, kPerLeaf, acceptDocs, Integer.MAX_VALUE);
+    } else {
+      BitSetIterator filterIterator = filterCollector.getIterator(ctx.ord);
+      if (filterIterator == null || filterIterator.cost() == 0) {
+        return NO_RESULTS;
+      }
+
+      if (filterIterator.cost() <= k) {
+        // If there <= k possible matches, short-circuit and perform exact search, since HNSW must
+        // always visit at least k documents
+        return exactSearch(ctx, target, k, filterIterator);
+      }
+
+      try {
+        // The filter iterator already incorporates live docs
+        Bits acceptDocs = filterIterator.getBitSet();
+        int visitedLimit = (int) filterIterator.cost();
+        return ctx.reader().searchNearestVectors(field, target, kPerLeaf, acceptDocs, visitedLimit);
+      } catch (
+          @SuppressWarnings("unused")
+          CollectionTerminatedException e) {
+        // We stopped the kNN search because it visited too many nodes, so fall back to exact search
+        return exactSearch(ctx, target, k, filterIterator);
+      }
     }
+  }
 
-    TopDocs results = ctx.reader().searchNearestVectors(field, target, kPerLeaf, bitsFilter);
-    if (results == null) {
+  private TopDocs exactSearch(
+      LeafReaderContext context, float[] target, int k, DocIdSetIterator acceptIterator)
+      throws IOException {
+    FieldInfo fi = context.reader().getFieldInfos().fieldInfo(field);
+    if (fi == null || fi.getVectorDimension() == 0) {
+      // The field does not exist or does not index vectors
       return NO_RESULTS;
     }
-    if (ctx.docBase > 0) {
-      for (ScoreDoc scoreDoc : results.scoreDocs) {
-        scoreDoc.doc += ctx.docBase;
-      }
+
+    VectorSimilarityFunction similarityFunction = fi.getVectorSimilarityFunction();
+    VectorValues vectorValues = context.reader().getVectorValues(field);
+
+    HitQueue queue = new HitQueue(k, false);

Review comment:
       Did you consider using the pre-populated version? We might be creating and discarding a lot of `ScoreDoc`s here.

##########
File path: lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java
##########
@@ -96,43 +107,98 @@ public Query rewrite(IndexReader reader) throws IOException {
     return createRewrittenQuery(reader, topK);
   }
 
-  private TopDocs searchLeaf(LeafReaderContext ctx, int kPerLeaf, Bits bitsFilter)
+  private TopDocs searchLeaf(LeafReaderContext ctx, int kPerLeaf, BitSetCollector filterCollector)
       throws IOException {
-    // If the filter is non-null, then it already handles live docs
-    if (bitsFilter == null) {
-      bitsFilter = ctx.reader().getLiveDocs();
+
+    if (filterCollector == null) {
+      Bits acceptDocs = ctx.reader().getLiveDocs();
+      return ctx.reader()
+          .searchNearestVectors(field, target, kPerLeaf, acceptDocs, Integer.MAX_VALUE);
+    } else {
+      BitSetIterator filterIterator = filterCollector.getIterator(ctx.ord);
+      if (filterIterator == null || filterIterator.cost() == 0) {

Review comment:
       A first I was worried about treating `0` as a hard filter since `cost` is only heuristic, generally, but in this case the `filterCollector` was produced through exhaustive iteration so its `cost` is exact and this is OK.

##########
File path: lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java
##########
@@ -96,43 +107,98 @@ public Query rewrite(IndexReader reader) throws IOException {
     return createRewrittenQuery(reader, topK);
   }
 
-  private TopDocs searchLeaf(LeafReaderContext ctx, int kPerLeaf, Bits bitsFilter)
+  private TopDocs searchLeaf(LeafReaderContext ctx, int kPerLeaf, BitSetCollector filterCollector)
       throws IOException {
-    // If the filter is non-null, then it already handles live docs
-    if (bitsFilter == null) {
-      bitsFilter = ctx.reader().getLiveDocs();
+
+    if (filterCollector == null) {
+      Bits acceptDocs = ctx.reader().getLiveDocs();
+      return ctx.reader()
+          .searchNearestVectors(field, target, kPerLeaf, acceptDocs, Integer.MAX_VALUE);
+    } else {
+      BitSetIterator filterIterator = filterCollector.getIterator(ctx.ord);
+      if (filterIterator == null || filterIterator.cost() == 0) {
+        return NO_RESULTS;
+      }
+
+      if (filterIterator.cost() <= k) {
+        // If there <= k possible matches, short-circuit and perform exact search, since HNSW must
+        // always visit at least k documents
+        return exactSearch(ctx, target, k, filterIterator);
+      }
+
+      try {
+        // The filter iterator already incorporates live docs
+        Bits acceptDocs = filterIterator.getBitSet();
+        int visitedLimit = (int) filterIterator.cost();
+        return ctx.reader().searchNearestVectors(field, target, kPerLeaf, acceptDocs, visitedLimit);

Review comment:
       Oh, I see, annoyingly `searchNearestVectors` can return `null`, and that is endemic in `CodecReader` :) We could add a helper function here `emptyIfNull()` or so, but it's purely cosmetic, not really needed.

##########
File path: lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java
##########
@@ -96,43 +107,98 @@ public Query rewrite(IndexReader reader) throws IOException {
     return createRewrittenQuery(reader, topK);
   }
 
-  private TopDocs searchLeaf(LeafReaderContext ctx, int kPerLeaf, Bits bitsFilter)
+  private TopDocs searchLeaf(LeafReaderContext ctx, int kPerLeaf, BitSetCollector filterCollector)
       throws IOException {
-    // If the filter is non-null, then it already handles live docs
-    if (bitsFilter == null) {
-      bitsFilter = ctx.reader().getLiveDocs();
+
+    if (filterCollector == null) {
+      Bits acceptDocs = ctx.reader().getLiveDocs();
+      return ctx.reader()
+          .searchNearestVectors(field, target, kPerLeaf, acceptDocs, Integer.MAX_VALUE);
+    } else {
+      BitSetIterator filterIterator = filterCollector.getIterator(ctx.ord);
+      if (filterIterator == null || filterIterator.cost() == 0) {
+        return NO_RESULTS;
+      }
+
+      if (filterIterator.cost() <= k) {
+        // If there <= k possible matches, short-circuit and perform exact search, since HNSW must
+        // always visit at least k documents
+        return exactSearch(ctx, target, k, filterIterator);
+      }
+
+      try {
+        // The filter iterator already incorporates live docs
+        Bits acceptDocs = filterIterator.getBitSet();
+        int visitedLimit = (int) filterIterator.cost();
+        return ctx.reader().searchNearestVectors(field, target, kPerLeaf, acceptDocs, visitedLimit);
+      } catch (
+          @SuppressWarnings("unused")
+          CollectionTerminatedException e) {
+        // We stopped the kNN search because it visited too many nodes, so fall back to exact search
+        return exactSearch(ctx, target, k, filterIterator);
+      }
     }
+  }
 
-    TopDocs results = ctx.reader().searchNearestVectors(field, target, kPerLeaf, bitsFilter);
-    if (results == null) {
+  private TopDocs exactSearch(
+      LeafReaderContext context, float[] target, int k, DocIdSetIterator acceptIterator)
+      throws IOException {
+    FieldInfo fi = context.reader().getFieldInfos().fieldInfo(field);
+    if (fi == null || fi.getVectorDimension() == 0) {
+      // The field does not exist or does not index vectors
       return NO_RESULTS;
     }
-    if (ctx.docBase > 0) {
-      for (ScoreDoc scoreDoc : results.scoreDocs) {
-        scoreDoc.doc += ctx.docBase;
-      }
+
+    VectorSimilarityFunction similarityFunction = fi.getVectorSimilarityFunction();
+    VectorValues vectorValues = context.reader().getVectorValues(field);
+
+    HitQueue queue = new HitQueue(k, false);
+    DocIdSetIterator iterator =
+        ConjunctionUtils.intersectIterators(List.of(acceptIterator, vectorValues));
+
+    int doc;
+    while ((doc = iterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+      float[] vector = vectorValues.vectorValue();
+      float score = similarityFunction.convertToScore(similarityFunction.compare(vector, target));
+      queue.insertWithOverflow(new ScoreDoc(doc, score));
     }
-    return results;
+    ScoreDoc[] topScoreDocs = new ScoreDoc[queue.size()];
+    for (int i = topScoreDocs.length - 1; i >= 0; i--) {
+      topScoreDocs[i] = queue.pop();
+    }
+
+    TotalHits totalHits = new TotalHits(acceptIterator.cost(), TotalHits.Relation.EQUAL_TO);

Review comment:
       Right, we can report `EQUAL_TO` since we exhaustively iterated

##########
File path: lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java
##########
@@ -96,43 +107,98 @@ public Query rewrite(IndexReader reader) throws IOException {
     return createRewrittenQuery(reader, topK);
   }
 
-  private TopDocs searchLeaf(LeafReaderContext ctx, int kPerLeaf, Bits bitsFilter)
+  private TopDocs searchLeaf(LeafReaderContext ctx, int kPerLeaf, BitSetCollector filterCollector)
       throws IOException {
-    // If the filter is non-null, then it already handles live docs
-    if (bitsFilter == null) {
-      bitsFilter = ctx.reader().getLiveDocs();
+
+    if (filterCollector == null) {
+      Bits acceptDocs = ctx.reader().getLiveDocs();
+      return ctx.reader()
+          .searchNearestVectors(field, target, kPerLeaf, acceptDocs, Integer.MAX_VALUE);
+    } else {
+      BitSetIterator filterIterator = filterCollector.getIterator(ctx.ord);
+      if (filterIterator == null || filterIterator.cost() == 0) {
+        return NO_RESULTS;
+      }
+
+      if (filterIterator.cost() <= k) {
+        // If there <= k possible matches, short-circuit and perform exact search, since HNSW must
+        // always visit at least k documents
+        return exactSearch(ctx, target, k, filterIterator);
+      }
+
+      try {
+        // The filter iterator already incorporates live docs
+        Bits acceptDocs = filterIterator.getBitSet();
+        int visitedLimit = (int) filterIterator.cost();
+        return ctx.reader().searchNearestVectors(field, target, kPerLeaf, acceptDocs, visitedLimit);
+      } catch (
+          @SuppressWarnings("unused")
+          CollectionTerminatedException e) {

Review comment:
       I could go either way with this one. I tend to lean towards using `TopDocs.totalHits.value` since we already use it for returning visited counts; we could return with a null or maybe empty `scoreDocs` in that case? Or perhaps there could be a use case for returning the "best effort" results obtained by visiting a limited subset of the graph, and we should in fact marshal up the results. Generally I don't favor using Exceptions for expected behavior, but also I think if we do choose this pattern we should create a new Exception type just for this case.

##########
File path: lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java
##########
@@ -76,17 +81,23 @@ public KnnVectorQuery(String field, float[] target, int k, Query filter) {
 
   @Override
   public Query rewrite(IndexReader reader) throws IOException {
-    BitSet[] bitSets = null;
+    TopDocs[] perLeafResults = new TopDocs[reader.leaves().size()];
 
+    BitSetCollector filterCollector = null;
     if (filter != null) {
+      filterCollector = new BitSetCollector(reader.leaves().size());
       IndexSearcher indexSearcher = new IndexSearcher(reader);
-      bitSets = new BitSet[reader.leaves().size()];
-      indexSearcher.search(filter, new BitSetCollector(bitSets));
+      indexSearcher.search(filter, filterCollector);
     }
 
-    TopDocs[] perLeafResults = new TopDocs[reader.leaves().size()];
     for (LeafReaderContext ctx : reader.leaves()) {
-      perLeafResults[ctx.ord] = searchLeaf(ctx, k, bitSets != null ? bitSets[ctx.ord] : null);
+      TopDocs results = searchLeaf(ctx, k, filterCollector);
+      if (results != null && ctx.docBase > 0) {
+        for (ScoreDoc scoreDoc : results.scoreDocs) {
+          scoreDoc.doc += ctx.docBase;
+        }
+      }
+      perLeafResults[ctx.ord] = results != null ? results : NO_RESULTS;

Review comment:
       can `searchLeaf` actually return `null`? Should we make it return `NO_RESULTS` then?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] msokolov commented on a change in pull request #656: LUCENE-10382: Support filtering in KnnVectorQuery

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #656:
URL: https://github.com/apache/lucene/pull/656#discussion_r806991813



##########
File path: lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java
##########
@@ -70,18 +121,125 @@ public Query rewrite(IndexReader reader) throws IOException {
     return createRewrittenQuery(reader, topK);
   }
 
-  private TopDocs searchLeaf(LeafReaderContext ctx, int kPerLeaf) throws IOException {
-    Bits liveDocs = ctx.reader().getLiveDocs();
-    TopDocs results = ctx.reader().searchNearestVectors(field, target, kPerLeaf, liveDocs);
-    if (results == null) {
+  private TopDocs searchLeaf(LeafReaderContext ctx, BitSetCollector filterCollector)
+      throws IOException {
+
+    if (filterCollector == null) {
+      Bits acceptDocs = ctx.reader().getLiveDocs();
+      return approximateSearch(ctx, acceptDocs, Integer.MAX_VALUE);
+    } else {
+      BitSetIterator filterIterator = filterCollector.getIterator(ctx.ord);
+      if (filterIterator == null || filterIterator.cost() == 0) {
+        return NO_RESULTS;
+      }
+
+      if (filterIterator.cost() <= k) {
+        // If there <= k possible matches, short-circuit and perform exact search, since HNSW must

Review comment:
       "If there are"

##########
File path: lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsReader.java
##########
@@ -58,6 +60,12 @@ protected KnnVectorsReader() {}
    * true k closest neighbors. For large values of k (for example when k is close to the total
    * number of documents), the search may also retrieve fewer than k documents.
    *
+   * <p>The returned {@link TopDocs} will contain a {@link ScoreDoc} for each nearest neighbor, in
+   * order of their similarity to the query vector (decreasing scores). The {@link TotalHits}
+   * contains the number of documents visited during the search. If the search stopped early because
+   * it hit {@code visitedLimit}, it is indicated through the relation {@code

Review comment:
       Would it be enough to know that `TopDocs.totalHits.value==visitedLimit`? Do we need to use the relation as a sentinel?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jtibshirani commented on a change in pull request #656: LUCENE-10382: Support filtering in KnnVectorQuery

Posted by GitBox <gi...@apache.org>.
jtibshirani commented on a change in pull request #656:
URL: https://github.com/apache/lucene/pull/656#discussion_r807399758



##########
File path: lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsReader.java
##########
@@ -58,6 +60,12 @@ protected KnnVectorsReader() {}
    * true k closest neighbors. For large values of k (for example when k is close to the total
    * number of documents), the search may also retrieve fewer than k documents.
    *
+   * <p>The returned {@link TopDocs} will contain a {@link ScoreDoc} for each nearest neighbor, in
+   * order of their similarity to the query vector (decreasing scores). The {@link TotalHits}
+   * contains the number of documents visited during the search. If the search stopped early because
+   * it hit {@code visitedLimit}, it is indicated through the relation {@code

Review comment:
       It's possible (but unlikely) that the search completed normally with exactly `numVisited == visitedLimit`. The `visitedLimit` is inclusive. To me it felt more solid and obvious to use the relation.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #656: LUCENE-10382: Support filtering in KnnVectorQuery

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #656:
URL: https://github.com/apache/lucene/pull/656#discussion_r808862885



##########
File path: lucene/core/src/test/org/apache/lucene/search/TestKnnVectorQuery.java
##########
@@ -455,6 +484,61 @@ public void testRandom() throws IOException {
     }
   }
 
+  /** Tests with random vectors and a random filter. Uses RandomIndexWriter. */
+  public void testRandomWithFilter() throws IOException {
+    int numDocs = 200;
+    int dimension = atLeast(5);
+    int numIters = atLeast(10);
+    try (Directory d = newDirectory()) {
+      RandomIndexWriter w = new RandomIndexWriter(random(), d);
+      for (int i = 0; i < numDocs; i++) {
+        Document doc = new Document();
+        doc.add(new KnnVectorField("field", randomVector(dimension)));
+        doc.add(new NumericDocValuesField("tag", i));
+        doc.add(new IntPoint("tag", i));
+        w.addDocument(doc);
+      }
+      w.close();
+
+      try (IndexReader reader = DirectoryReader.open(d)) {
+        IndexSearcher searcher = newSearcher(reader);
+        for (int i = 0; i < numIters; i++) {
+          int lower = random().nextInt(50);
+
+          // Check that when filter is restrictive, we use exact search
+          Query filter = IntPoint.newRangeQuery("tag", lower, lower + 6);
+          KnnVectorQuery query = new KnnVectorQuery("field", randomVector(dimension), 5, filter);
+          TopDocs results = searcher.search(query, numDocs);
+          assertEquals(TotalHits.Relation.EQUAL_TO, results.totalHits.relation);
+          assertEquals(results.totalHits.value, 5);

Review comment:
       Thanks for the explanation, I missed a part about rewriting to  `DocAndScoreQuery`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jtibshirani commented on a change in pull request #656: LUCENE-10382: Support filtering in KnnVectorQuery

Posted by GitBox <gi...@apache.org>.
jtibshirani commented on a change in pull request #656:
URL: https://github.com/apache/lucene/pull/656#discussion_r803950304



##########
File path: lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java
##########
@@ -96,43 +107,98 @@ public Query rewrite(IndexReader reader) throws IOException {
     return createRewrittenQuery(reader, topK);
   }
 
-  private TopDocs searchLeaf(LeafReaderContext ctx, int kPerLeaf, Bits bitsFilter)
+  private TopDocs searchLeaf(LeafReaderContext ctx, int kPerLeaf, BitSetCollector filterCollector)
       throws IOException {
-    // If the filter is non-null, then it already handles live docs
-    if (bitsFilter == null) {
-      bitsFilter = ctx.reader().getLiveDocs();
+
+    if (filterCollector == null) {
+      Bits acceptDocs = ctx.reader().getLiveDocs();
+      return ctx.reader()
+          .searchNearestVectors(field, target, kPerLeaf, acceptDocs, Integer.MAX_VALUE);
+    } else {
+      BitSetIterator filterIterator = filterCollector.getIterator(ctx.ord);
+      if (filterIterator == null || filterIterator.cost() == 0) {
+        return NO_RESULTS;
+      }
+
+      if (filterIterator.cost() <= k) {
+        // If there <= k possible matches, short-circuit and perform exact search, since HNSW must
+        // always visit at least k documents
+        return exactSearch(ctx, target, k, filterIterator);
+      }
+
+      try {
+        // The filter iterator already incorporates live docs
+        Bits acceptDocs = filterIterator.getBitSet();
+        int visitedLimit = (int) filterIterator.cost();
+        return ctx.reader().searchNearestVectors(field, target, kPerLeaf, acceptDocs, visitedLimit);
+      } catch (
+          @SuppressWarnings("unused")
+          CollectionTerminatedException e) {
+        // We stopped the kNN search because it visited too many nodes, so fall back to exact search
+        return exactSearch(ctx, target, k, filterIterator);
+      }
     }
+  }
 
-    TopDocs results = ctx.reader().searchNearestVectors(field, target, kPerLeaf, bitsFilter);
-    if (results == null) {
+  private TopDocs exactSearch(
+      LeafReaderContext context, float[] target, int k, DocIdSetIterator acceptIterator)
+      throws IOException {
+    FieldInfo fi = context.reader().getFieldInfos().fieldInfo(field);
+    if (fi == null || fi.getVectorDimension() == 0) {
+      // The field does not exist or does not index vectors
       return NO_RESULTS;
     }
-    if (ctx.docBase > 0) {
-      for (ScoreDoc scoreDoc : results.scoreDocs) {
-        scoreDoc.doc += ctx.docBase;
-      }
+
+    VectorSimilarityFunction similarityFunction = fi.getVectorSimilarityFunction();
+    VectorValues vectorValues = context.reader().getVectorValues(field);
+
+    HitQueue queue = new HitQueue(k, false);

Review comment:
       Oh this is good to know about, I'll try to switch over.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jtibshirani commented on a change in pull request #656: LUCENE-10382: Support filtering in KnnVectorQuery

Posted by GitBox <gi...@apache.org>.
jtibshirani commented on a change in pull request #656:
URL: https://github.com/apache/lucene/pull/656#discussion_r804009517



##########
File path: lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java
##########
@@ -70,18 +118,104 @@ public Query rewrite(IndexReader reader) throws IOException {
     return createRewrittenQuery(reader, topK);
   }
 
-  private TopDocs searchLeaf(LeafReaderContext ctx, int kPerLeaf) throws IOException {
-    Bits liveDocs = ctx.reader().getLiveDocs();
-    TopDocs results = ctx.reader().searchNearestVectors(field, target, kPerLeaf, liveDocs);
-    if (results == null) {
+  private TopDocs searchLeaf(LeafReaderContext ctx, int kPerLeaf, BitSetCollector filterCollector)
+      throws IOException {
+
+    if (filterCollector == null) {
+      Bits acceptDocs = ctx.reader().getLiveDocs();
+      return ctx.reader()
+          .searchNearestVectors(field, target, kPerLeaf, acceptDocs, Integer.MAX_VALUE);
+    } else {
+      BitSetIterator filterIterator = filterCollector.getIterator(ctx.ord);
+      if (filterIterator == null || filterIterator.cost() == 0) {
+        return NO_RESULTS;
+      }
+
+      if (filterIterator.cost() <= k) {
+        // If there <= k possible matches, short-circuit and perform exact search, since HNSW must
+        // always visit at least k documents
+        return exactSearch(ctx, target, k, filterIterator);
+      }
+
+      try {
+        // The filter iterator already incorporates live docs
+        Bits acceptDocs = filterIterator.getBitSet();
+        int visitedLimit = (int) filterIterator.cost();
+        return ctx.reader().searchNearestVectors(field, target, kPerLeaf, acceptDocs, visitedLimit);
+      } catch (
+          @SuppressWarnings("unused")
+          CollectionTerminatedException e) {
+        // We stopped the kNN search because it visited too many nodes, so fall back to exact search
+        return exactSearch(ctx, target, k, filterIterator);
+      }
+    }
+  }
+
+  private TopDocs exactSearch(
+      LeafReaderContext context, float[] target, int k, DocIdSetIterator acceptIterator)
+      throws IOException {
+    FieldInfo fi = context.reader().getFieldInfos().fieldInfo(field);
+    if (fi == null || fi.getVectorDimension() == 0) {
+      // The field does not exist or does not index vectors
       return NO_RESULTS;
     }
-    if (ctx.docBase > 0) {
-      for (ScoreDoc scoreDoc : results.scoreDocs) {
-        scoreDoc.doc += ctx.docBase;
+
+    VectorSimilarityFunction similarityFunction = fi.getVectorSimilarityFunction();
+    VectorValues vectorValues = context.reader().getVectorValues(field);
+
+    HitQueue queue = new HitQueue(k, false);
+    DocIdSetIterator iterator =
+        ConjunctionUtils.intersectIterators(List.of(acceptIterator, vectorValues));

Review comment:
       I just noticed: maybe we should move this intersection earlier to when we execute the filter into a bitset. The way we do it now, our assessment of the filter selectivity is inaccurate when docs are missing vectors.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jtibshirani commented on pull request #656: LUCENE-10382: Support filtering in KnnVectorQuery

Posted by GitBox <gi...@apache.org>.
jtibshirani commented on pull request #656:
URL: https://github.com/apache/lucene/pull/656#issuecomment-1032109021


   I tried out the around stopping the HNSW search early if it visits too many docs. To test, I modified `KnnGraphTester` to create `acceptDocs` uniformly at random with a certain selectivity, then measured recall and QPS. Here are the results on glove-100-angular (~1.2 million docs) with a filter selectivity 0.01:
   
   **Baseline**
   ```
   k        Recall    VisitedDocs     QPS  
   
   10        0.774       15957     232.083
   50        0.930       63429      58.994
   80        0.958       95175      42.470
   100       0.967      118891      35.203
   500       0.997     1176237       8.136
   800       0.999     1183514       5.571
   ```
   
   **PR**
   ```
   k        Recall    VisitedDocs     QPS  
   10        1.000	       22908     190.286
   50        1.000	       23607     152.224
   80        1.000	       23608     148.036
   100       1.000	       23608     145.381
   500       1.000	       23608     138.903
   800       1.000	       23608     137.882
   ```
   
   Since the filter is so selective, HNSW always visits more than 1% of the docs. The adaptive logic in the PR decides to stop the search and switch to an exact search, which bounds the visited docs at 2%. For `k=10` this makes the QPS a little worse, but overall prevents QPS from degrading (with the side benefit of perfect recall). I also tested with less restrictive filters, and in these cases the fallback just doesn't kick in, so the QPS remains the same as before.
   
   Overall I like this approach because it doesn't require us to fiddle with thresholds or expose new parameters. It could also help make HNSW more robust in "pathological" cases where even when the filter is not that selective, all the nearest vectors to a query happen to be filtered away.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jtibshirani commented on a change in pull request #656: LUCENE-10382: Support filtering in KnnVectorQuery

Posted by GitBox <gi...@apache.org>.
jtibshirani commented on a change in pull request #656:
URL: https://github.com/apache/lucene/pull/656#discussion_r803965732



##########
File path: lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java
##########
@@ -96,43 +107,98 @@ public Query rewrite(IndexReader reader) throws IOException {
     return createRewrittenQuery(reader, topK);
   }
 
-  private TopDocs searchLeaf(LeafReaderContext ctx, int kPerLeaf, Bits bitsFilter)
+  private TopDocs searchLeaf(LeafReaderContext ctx, int kPerLeaf, BitSetCollector filterCollector)
       throws IOException {
-    // If the filter is non-null, then it already handles live docs
-    if (bitsFilter == null) {
-      bitsFilter = ctx.reader().getLiveDocs();
+
+    if (filterCollector == null) {
+      Bits acceptDocs = ctx.reader().getLiveDocs();
+      return ctx.reader()
+          .searchNearestVectors(field, target, kPerLeaf, acceptDocs, Integer.MAX_VALUE);
+    } else {
+      BitSetIterator filterIterator = filterCollector.getIterator(ctx.ord);
+      if (filterIterator == null || filterIterator.cost() == 0) {
+        return NO_RESULTS;
+      }
+
+      if (filterIterator.cost() <= k) {
+        // If there <= k possible matches, short-circuit and perform exact search, since HNSW must
+        // always visit at least k documents
+        return exactSearch(ctx, target, k, filterIterator);
+      }
+
+      try {
+        // The filter iterator already incorporates live docs
+        Bits acceptDocs = filterIterator.getBitSet();
+        int visitedLimit = (int) filterIterator.cost();
+        return ctx.reader().searchNearestVectors(field, target, kPerLeaf, acceptDocs, visitedLimit);
+      } catch (
+          @SuppressWarnings("unused")
+          CollectionTerminatedException e) {

Review comment:
       I agree, it's nice to avoid using exceptions for normal control flow. I'm not too concerned from a performance perspective though, exceptions aren't thrown in a "hot loop" and I didn't see a perf hit in testing.
   
   If we go the route of using `TopDocs`, I'd prefer to avoid 'null' since that's a bit overloaded (indicates the field is missing or does not have vectors). Brainstorming ideas:
   * Just return `EMPTY_TOPDOCS`.
   * Still return best score docs and the visited count. But use `EQUAL_TO` for `TotalHits.Relation` if the search completed normally, otherwise use `GREATER_THAN_OR_EQUAL_TO`. 
   * Use a special subtype of `TopDocs` instead, which has an explicit "complete" flag?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #656: LUCENE-10382: Support filtering in KnnVectorQuery

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #656:
URL: https://github.com/apache/lucene/pull/656#discussion_r807653039



##########
File path: lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java
##########
@@ -24,19 +24,36 @@
 import java.util.Objects;
 import org.apache.lucene.codecs.KnnVectorsReader;
 import org.apache.lucene.document.KnnVectorField;
+import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.VectorSimilarityFunction;
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.util.BitSet;
+import org.apache.lucene.util.BitSetIterator;
 import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.FixedBitSet;
 
-/** Uses {@link KnnVectorsReader#search} to perform nearest neighbour search. */
+/**
+ * Uses {@link KnnVectorsReader#search} to perform nearest neighbour search.
+ *
+ * <p>This query also allows for performing a kNN search subject to a filter. In this case, it first
+ * executes the filter for each leaf, then chooses a strategy dynamically:
+ *
+ * <ul>
+ *   <li>If the filter cost is less than k, just execute an exact search
+ *   <li>Otherwise run a kNN search subject to the filter
+ *   <li>the kNN search visits too many vectors without completing, stop and run an exact search

Review comment:
       **if** the KNN search ?

##########
File path: lucene/core/src/test/org/apache/lucene/search/TestKnnVectorQuery.java
##########
@@ -455,6 +484,61 @@ public void testRandom() throws IOException {
     }
   }
 
+  /** Tests with random vectors and a random filter. Uses RandomIndexWriter. */
+  public void testRandomWithFilter() throws IOException {
+    int numDocs = 200;
+    int dimension = atLeast(5);
+    int numIters = atLeast(10);
+    try (Directory d = newDirectory()) {
+      RandomIndexWriter w = new RandomIndexWriter(random(), d);
+      for (int i = 0; i < numDocs; i++) {
+        Document doc = new Document();
+        doc.add(new KnnVectorField("field", randomVector(dimension)));
+        doc.add(new NumericDocValuesField("tag", i));
+        doc.add(new IntPoint("tag", i));
+        w.addDocument(doc);
+      }
+      w.close();
+
+      try (IndexReader reader = DirectoryReader.open(d)) {
+        IndexSearcher searcher = newSearcher(reader);
+        for (int i = 0; i < numIters; i++) {
+          int lower = random().nextInt(50);
+
+          // Check that when filter is restrictive, we use exact search
+          Query filter = IntPoint.newRangeQuery("tag", lower, lower + 6);
+          KnnVectorQuery query = new KnnVectorQuery("field", randomVector(dimension), 5, filter);
+          TopDocs results = searcher.search(query, numDocs);
+          assertEquals(TotalHits.Relation.EQUAL_TO, results.totalHits.relation);
+          assertEquals(results.totalHits.value, 5);

Review comment:
       How do we know that we used the exact search?  Are we judging by the equality of `results.totalHits.value` and `results.scoreDocs.length`?  I guess in most cases this is true.
   
   Another idea is always use `TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO` for the approximate search results as returned in `KnnVectorQuery.searchLeaf`:
   ```java
   TopDocs results = approximateSearch(ctx, acceptDocs, visitedLimit);
         if (results.totalHits.relation == TotalHits.Relation.EQUAL_TO) {
           return <results with Relation.GREATER_THAN_OR_EQUAL_TO>;
         } else {
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jtibshirani commented on a change in pull request #656: LUCENE-10382: Support filtering in KnnVectorQuery

Posted by GitBox <gi...@apache.org>.
jtibshirani commented on a change in pull request #656:
URL: https://github.com/apache/lucene/pull/656#discussion_r808356718



##########
File path: lucene/core/src/test/org/apache/lucene/search/TestKnnVectorQuery.java
##########
@@ -455,6 +484,61 @@ public void testRandom() throws IOException {
     }
   }
 
+  /** Tests with random vectors and a random filter. Uses RandomIndexWriter. */
+  public void testRandomWithFilter() throws IOException {
+    int numDocs = 200;
+    int dimension = atLeast(5);
+    int numIters = atLeast(10);
+    try (Directory d = newDirectory()) {
+      RandomIndexWriter w = new RandomIndexWriter(random(), d);
+      for (int i = 0; i < numDocs; i++) {
+        Document doc = new Document();
+        doc.add(new KnnVectorField("field", randomVector(dimension)));
+        doc.add(new NumericDocValuesField("tag", i));
+        doc.add(new IntPoint("tag", i));
+        w.addDocument(doc);
+      }
+      w.close();
+
+      try (IndexReader reader = DirectoryReader.open(d)) {
+        IndexSearcher searcher = newSearcher(reader);
+        for (int i = 0; i < numIters; i++) {
+          int lower = random().nextInt(50);
+
+          // Check that when filter is restrictive, we use exact search
+          Query filter = IntPoint.newRangeQuery("tag", lower, lower + 6);
+          KnnVectorQuery query = new KnnVectorQuery("field", randomVector(dimension), 5, filter);
+          TopDocs results = searcher.search(query, numDocs);
+          assertEquals(TotalHits.Relation.EQUAL_TO, results.totalHits.relation);
+          assertEquals(results.totalHits.value, 5);

Review comment:
       Thanks for catching this. I actually got confused here and wrote test assertions that are misleading. Since `KnnVectorQuery` is rewritten to `DocAndScoreQuery`, none of the information about visited nodes is preserved. Therefore we can't tell if exact or approximate search was used. I will rework this test.
   
   I will open a follow-up issue to discuss this. I don't feel like we have a perfect grasp on what total hits should mean in the context of kNN search, especially since it differs between `LeafReader#searchNearestVectors` and the output of `KnnVectorQuery`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jtibshirani commented on a change in pull request #656: LUCENE-10382: Support filtering in KnnVectorQuery

Posted by GitBox <gi...@apache.org>.
jtibshirani commented on a change in pull request #656:
URL: https://github.com/apache/lucene/pull/656#discussion_r808308244



##########
File path: build.gradle
##########
@@ -183,3 +183,5 @@ apply from: file('gradle/hacks/turbocharge-jvm-opts.gradle')
 apply from: file('gradle/hacks/dummy-outputs.gradle')
 
 apply from: file('gradle/pylucene/pylucene.gradle')
+sourceCompatibility = JavaVersion.VERSION_16

Review comment:
       Definitely not! Somehow this file gets automatically changed, and I accidentally included it with `git add -u`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] msokolov commented on a change in pull request #656: LUCENE-10382: Support filtering in KnnVectorQuery

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #656:
URL: https://github.com/apache/lucene/pull/656#discussion_r806991813



##########
File path: lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java
##########
@@ -70,18 +121,125 @@ public Query rewrite(IndexReader reader) throws IOException {
     return createRewrittenQuery(reader, topK);
   }
 
-  private TopDocs searchLeaf(LeafReaderContext ctx, int kPerLeaf) throws IOException {
-    Bits liveDocs = ctx.reader().getLiveDocs();
-    TopDocs results = ctx.reader().searchNearestVectors(field, target, kPerLeaf, liveDocs);
-    if (results == null) {
+  private TopDocs searchLeaf(LeafReaderContext ctx, BitSetCollector filterCollector)
+      throws IOException {
+
+    if (filterCollector == null) {
+      Bits acceptDocs = ctx.reader().getLiveDocs();
+      return approximateSearch(ctx, acceptDocs, Integer.MAX_VALUE);
+    } else {
+      BitSetIterator filterIterator = filterCollector.getIterator(ctx.ord);
+      if (filterIterator == null || filterIterator.cost() == 0) {
+        return NO_RESULTS;
+      }
+
+      if (filterIterator.cost() <= k) {
+        // If there <= k possible matches, short-circuit and perform exact search, since HNSW must

Review comment:
       "If there are"

##########
File path: lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsReader.java
##########
@@ -58,6 +60,12 @@ protected KnnVectorsReader() {}
    * true k closest neighbors. For large values of k (for example when k is close to the total
    * number of documents), the search may also retrieve fewer than k documents.
    *
+   * <p>The returned {@link TopDocs} will contain a {@link ScoreDoc} for each nearest neighbor, in
+   * order of their similarity to the query vector (decreasing scores). The {@link TotalHits}
+   * contains the number of documents visited during the search. If the search stopped early because
+   * it hit {@code visitedLimit}, it is indicated through the relation {@code

Review comment:
       Would it be enough to know that `TopDocs.totalHits.value==visitedLimit`? Do we need to use the relation as a sentinel?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jtibshirani commented on a change in pull request #656: LUCENE-10382: Support filtering in KnnVectorQuery

Posted by GitBox <gi...@apache.org>.
jtibshirani commented on a change in pull request #656:
URL: https://github.com/apache/lucene/pull/656#discussion_r801192735



##########
File path: lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91HnswVectorsReader.java
##########
@@ -227,16 +231,36 @@ public TopDocs search(String field, float[] target, int k, Bits acceptDocs) thro
 
     // bound k by total number of vectors to prevent oversizing data structures
     k = Math.min(k, fieldEntry.size());
-
     OffHeapVectorValues vectorValues = getOffHeapVectorValues(fieldEntry);
+
+    DocIdSetIterator acceptIterator = null;
+    int visitedLimit = Integer.MAX_VALUE;
+
+    if (acceptDocs instanceof BitSet acceptBitSet) {

Review comment:
       This is a temporary hack since I wasn't sure about the right design. I could see a couple possibilities:
   1. Add a new `BitSet filter` parameter to `searchNearestVectors`, keeping the fallback logic within the HNSW classes. 
   2. Add a new `int visitedLimit` parameter to `LeafReader#searchNearestVectors`. Pull the "exact search" logic up into `KnnVectorQuery`.
   
   Which option is better probably depends on how other algorithms would handle filtering (which I am not sure about), and also if we think `visitedLimit` is useful in other contexts.
   
   I also played around with having `searchNearestVectors` take a `Collector` and using `CollectionTerminatedException`... but couldn't really see how this made sense.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jpountz commented on a change in pull request #656: LUCENE-10382: Support filtering in KnnVectorQuery

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #656:
URL: https://github.com/apache/lucene/pull/656#discussion_r801354469



##########
File path: lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91HnswVectorsReader.java
##########
@@ -227,16 +231,36 @@ public TopDocs search(String field, float[] target, int k, Bits acceptDocs) thro
 
     // bound k by total number of vectors to prevent oversizing data structures
     k = Math.min(k, fieldEntry.size());
-
     OffHeapVectorValues vectorValues = getOffHeapVectorValues(fieldEntry);
+
+    DocIdSetIterator acceptIterator = null;
+    int visitedLimit = Integer.MAX_VALUE;
+
+    if (acceptDocs instanceof BitSet acceptBitSet) {

Review comment:
       I think I have a preference for option 2
    - This feels like a high-level query planning decision, which belongs more to the query API than to the codec API.
    - My gut feeling is that a limit on the number of considered candidates is something that would be generalizable to most NN algorithms.
    - Queries might have better options than a BitSet at times, e.g. if the filter is a `IndexSortSortedNumericDocValuesRangeQuery`, then you could have both a Bits and DocIdSetIterator view of the matches that do not require materializing a BitSet.
    - Vectors are currently not handled by `ExitableDirectoryReader`. Option 1 would require adding a BitSet wrapper, while we'd like to keep the number of sub classes of `BitSet` to exactly 2, a case that the JVM handles better. With option 2 we could go with just a `Bits` wrapper that would check the timeout whenever `Bit#get` is called?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jtibshirani edited a comment on pull request #656: LUCENE-10382: Support filtering in KnnVectorQuery

Posted by GitBox <gi...@apache.org>.
jtibshirani edited a comment on pull request #656:
URL: https://github.com/apache/lucene/pull/656#issuecomment-1032109021


   I tried out the around stopping the HNSW search early if it visits too many docs. To test, I modified `KnnGraphTester` to create `acceptDocs` uniformly at random with a certain selectivity, then measured recall and QPS. Here are the results on glove-100-angular (~1.2 million docs) with a filter selectivity 0.01:
   
   **Baseline**
   ```
   k        Recall    VisitedDocs     QPS  
   10        0.774       15957     232.083
   50        0.930       63429      58.994
   80        0.958       95175      42.470
   100       0.967      118891      35.203
   500       0.997     1176237       8.136
   800       0.999     1183514       5.571
   ```
   
   **PR**
   ```
   k        Recall    VisitedDocs     QPS  
   10        1.000	       22908     190.286
   50        1.000	       23607     152.224
   80        1.000	       23608     148.036
   100       1.000	       23608     145.381
   500       1.000	       23608     138.903
   800       1.000	       23608     137.882
   ```
   
   Since the filter is so selective, HNSW always visits more than 1% of the docs. The adaptive logic in the PR decides to stop the search and switch to an exact search, which bounds the visited docs at 2%. For `k=10` this makes the QPS a little worse, but overall prevents QPS from degrading (with the side benefit of perfect recall). I also tested with less restrictive filters, and in these cases the fallback just doesn't kick in, so the QPS remains the same as before.
   
   Overall I like this approach because it doesn't require us to fiddle with thresholds or expose new parameters. It could also help make HNSW more robust in "pathological" cases where even when the filter is not that selective, all the nearest vectors to a query happen to be filtered away.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jtibshirani commented on a change in pull request #656: LUCENE-10382: Support filtering in KnnVectorQuery

Posted by GitBox <gi...@apache.org>.
jtibshirani commented on a change in pull request #656:
URL: https://github.com/apache/lucene/pull/656#discussion_r803395995



##########
File path: lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphSearcher.java
##########
@@ -147,6 +165,11 @@ NeighborQueue searchLevel(
           continue;
         }
 
+        numVisited++;
+        if (numVisited > visitedLimit) {
+          throw new CollectionTerminatedException();

Review comment:
       This may be an abuse of `CollectionTerminatedException`. Another idea would be to try to pass back the information that the search was terminated early in `TopDocs.TotalHits` (but this also doesn't seem ideal).




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jtibshirani merged pull request #656: LUCENE-10382: Support filtering in KnnVectorQuery

Posted by GitBox <gi...@apache.org>.
jtibshirani merged pull request #656:
URL: https://github.com/apache/lucene/pull/656


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jtibshirani commented on a change in pull request #656: LUCENE-10382: Support filtering in KnnVectorQuery

Posted by GitBox <gi...@apache.org>.
jtibshirani commented on a change in pull request #656:
URL: https://github.com/apache/lucene/pull/656#discussion_r802075310



##########
File path: lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91HnswVectorsReader.java
##########
@@ -227,16 +231,36 @@ public TopDocs search(String field, float[] target, int k, Bits acceptDocs) thro
 
     // bound k by total number of vectors to prevent oversizing data structures
     k = Math.min(k, fieldEntry.size());
-
     OffHeapVectorValues vectorValues = getOffHeapVectorValues(fieldEntry);
+
+    DocIdSetIterator acceptIterator = null;
+    int visitedLimit = Integer.MAX_VALUE;
+
+    if (acceptDocs instanceof BitSet acceptBitSet) {

Review comment:
       These are all great points. The reasons to prefer option 2 make sense to me (although I'm also not clear on the best strategy for supporting ExitableDirectoryReader). I had a similar intuition to @msokolov that `visitedLimit` feels like a cost-tradeoff parameter similar to efSearch/ fanout... but I don't yet see how to bridge the gap between these two concepts.
   
   In any case, I feel pretty good about adding a parameter `visitedLimit` for now. The concept indeed seems general, we have room to further generalize it later if needed (maybe an approximate `costLimit`?) or revise it.

##########
File path: lucene/core/src/java/org/apache/lucene/codecs/lucene91/Lucene91HnswVectorsReader.java
##########
@@ -227,16 +231,36 @@ public TopDocs search(String field, float[] target, int k, Bits acceptDocs) thro
 
     // bound k by total number of vectors to prevent oversizing data structures
     k = Math.min(k, fieldEntry.size());
-
     OffHeapVectorValues vectorValues = getOffHeapVectorValues(fieldEntry);
+
+    DocIdSetIterator acceptIterator = null;
+    int visitedLimit = Integer.MAX_VALUE;
+
+    if (acceptDocs instanceof BitSet acceptBitSet) {

Review comment:
       These are all great points. The reasons to prefer option 2 make sense to me (although I'm also not clear on the best strategy for supporting ExitableDirectoryReader). I had a similar intuition to @msokolov that `visitedLimit` feels like a cost-tradeoff parameter similar to efSearch/ fanout... but I don't yet see how to bridge the gap between these two concepts.
   
   In any case, I feel pretty good about adding a parameter `visitedLimit` for now. The concept indeed seems general, and we have room to further generalize it later if needed (maybe an approximate `costLimit`?) or revise it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #656: LUCENE-10382: Support filtering in KnnVectorQuery

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #656:
URL: https://github.com/apache/lucene/pull/656#discussion_r803804328



##########
File path: lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java
##########
@@ -96,43 +107,98 @@ public Query rewrite(IndexReader reader) throws IOException {
     return createRewrittenQuery(reader, topK);
   }
 
-  private TopDocs searchLeaf(LeafReaderContext ctx, int kPerLeaf, Bits bitsFilter)
+  private TopDocs searchLeaf(LeafReaderContext ctx, int kPerLeaf, BitSetCollector filterCollector)
       throws IOException {
-    // If the filter is non-null, then it already handles live docs
-    if (bitsFilter == null) {
-      bitsFilter = ctx.reader().getLiveDocs();
+
+    if (filterCollector == null) {
+      Bits acceptDocs = ctx.reader().getLiveDocs();
+      return ctx.reader()
+          .searchNearestVectors(field, target, kPerLeaf, acceptDocs, Integer.MAX_VALUE);
+    } else {
+      BitSetIterator filterIterator = filterCollector.getIterator(ctx.ord);
+      if (filterIterator == null || filterIterator.cost() == 0) {
+        return NO_RESULTS;
+      }
+
+      if (filterIterator.cost() <= k) {
+        // If there <= k possible matches, short-circuit and perform exact search, since HNSW must
+        // always visit at least k documents
+        return exactSearch(ctx, target, k, filterIterator);
+      }
+
+      try {
+        // The filter iterator already incorporates live docs
+        Bits acceptDocs = filterIterator.getBitSet();
+        int visitedLimit = (int) filterIterator.cost();
+        return ctx.reader().searchNearestVectors(field, target, kPerLeaf, acceptDocs, visitedLimit);
+      } catch (
+          @SuppressWarnings("unused")
+          CollectionTerminatedException e) {

Review comment:
       I agree and also prefer not to throw an Exception if possible; it is an expensive operation to throw an Exception in comparison with just returning a value.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #656: LUCENE-10382: Support filtering in KnnVectorQuery

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #656:
URL: https://github.com/apache/lucene/pull/656#discussion_r803804328



##########
File path: lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java
##########
@@ -96,43 +107,98 @@ public Query rewrite(IndexReader reader) throws IOException {
     return createRewrittenQuery(reader, topK);
   }
 
-  private TopDocs searchLeaf(LeafReaderContext ctx, int kPerLeaf, Bits bitsFilter)
+  private TopDocs searchLeaf(LeafReaderContext ctx, int kPerLeaf, BitSetCollector filterCollector)
       throws IOException {
-    // If the filter is non-null, then it already handles live docs
-    if (bitsFilter == null) {
-      bitsFilter = ctx.reader().getLiveDocs();
+
+    if (filterCollector == null) {
+      Bits acceptDocs = ctx.reader().getLiveDocs();
+      return ctx.reader()
+          .searchNearestVectors(field, target, kPerLeaf, acceptDocs, Integer.MAX_VALUE);
+    } else {
+      BitSetIterator filterIterator = filterCollector.getIterator(ctx.ord);
+      if (filterIterator == null || filterIterator.cost() == 0) {
+        return NO_RESULTS;
+      }
+
+      if (filterIterator.cost() <= k) {
+        // If there <= k possible matches, short-circuit and perform exact search, since HNSW must
+        // always visit at least k documents
+        return exactSearch(ctx, target, k, filterIterator);
+      }
+
+      try {
+        // The filter iterator already incorporates live docs
+        Bits acceptDocs = filterIterator.getBitSet();
+        int visitedLimit = (int) filterIterator.cost();
+        return ctx.reader().searchNearestVectors(field, target, kPerLeaf, acceptDocs, visitedLimit);
+      } catch (
+          @SuppressWarnings("unused")
+          CollectionTerminatedException e) {

Review comment:
       I agree, also it is an expensive operation to throw an Exception in comparison with a just returning a value.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] mayya-sharipova commented on a change in pull request #656: LUCENE-10382: Support filtering in KnnVectorQuery

Posted by GitBox <gi...@apache.org>.
mayya-sharipova commented on a change in pull request #656:
URL: https://github.com/apache/lucene/pull/656#discussion_r804028845



##########
File path: lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java
##########
@@ -96,43 +107,98 @@ public Query rewrite(IndexReader reader) throws IOException {
     return createRewrittenQuery(reader, topK);
   }
 
-  private TopDocs searchLeaf(LeafReaderContext ctx, int kPerLeaf, Bits bitsFilter)
+  private TopDocs searchLeaf(LeafReaderContext ctx, int kPerLeaf, BitSetCollector filterCollector)
       throws IOException {
-    // If the filter is non-null, then it already handles live docs
-    if (bitsFilter == null) {
-      bitsFilter = ctx.reader().getLiveDocs();
+
+    if (filterCollector == null) {
+      Bits acceptDocs = ctx.reader().getLiveDocs();
+      return ctx.reader()
+          .searchNearestVectors(field, target, kPerLeaf, acceptDocs, Integer.MAX_VALUE);
+    } else {
+      BitSetIterator filterIterator = filterCollector.getIterator(ctx.ord);
+      if (filterIterator == null || filterIterator.cost() == 0) {
+        return NO_RESULTS;
+      }
+
+      if (filterIterator.cost() <= k) {
+        // If there <= k possible matches, short-circuit and perform exact search, since HNSW must
+        // always visit at least k documents
+        return exactSearch(ctx, target, k, filterIterator);
+      }
+
+      try {
+        // The filter iterator already incorporates live docs
+        Bits acceptDocs = filterIterator.getBitSet();
+        int visitedLimit = (int) filterIterator.cost();
+        return ctx.reader().searchNearestVectors(field, target, kPerLeaf, acceptDocs, visitedLimit);
+      } catch (
+          @SuppressWarnings("unused")
+          CollectionTerminatedException e) {

Review comment:
       I liked very much of "a special subtype of TopDocs instead, which has an explicit "complete" flag"




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jtibshirani commented on pull request #656: LUCENE-10382: Support filtering in KnnVectorQuery

Posted by GitBox <gi...@apache.org>.
jtibshirani commented on pull request #656:
URL: https://github.com/apache/lucene/pull/656#issuecomment-1039714498


   @msokolov @jpountz @mayya-sharipova this is ready for another look. Notable changes:
   * When computing the filter results, only include documents that actually contain a vector. This gives an accurate estimate of the filter selectivity. To support this I introduced `KnnVectorFieldExistsQuery`, which seemed useful in its own right.
   * I stopped using `CollectionTerminationException` to indicate that the search hit the visited limit. Instead, we pass the information in `TopDocs` through `TotalHits`. The value is always the number of visited docs, but the relation is `GREATER_THAN_OR_EQUAL_TO` iff the search stopped early. This is kind of arbitrary but felt natural -- I'm very open to suggestions here! It's a fairly low-level API and it's marked experimental, so there is also room to refine it later. This update does not change the output of `KnnVectorQuery`.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org