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/06/10 04:41:07 UTC

[GitHub] [lucene] kaivalnp opened a new pull request, #951: LUCENE-10606: Optimize Prefilter Hit Collection

kaivalnp opened a new pull request, #951:
URL: https://github.com/apache/lucene/pull/951

   ### Description
   
   Link to [Jira](https://issues.apache.org/jira/browse/LUCENE-10606)
   
   ### Solution
   
   Instead of collecting hit-by-hit using a `LeafCollector`, we break down the search by instantiating a weight, creating scorers, and checking the underlying iterator. If it is backed by a `BitSet`, we directly update the reference (as we won't be editing the `Bits`). Else we can create a new `BitSet` from the iterator using `BitSet.of`


-- 
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] kaivalnp commented on a diff in pull request #951: LUCENE-10606: Optimize Prefilter Hit Collection

Posted by GitBox <gi...@apache.org>.
kaivalnp commented on code in PR #951:
URL: https://github.com/apache/lucene/pull/951#discussion_r902366042


##########
lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java:
##########
@@ -92,20 +91,40 @@ public KnnVectorQuery(String field, float[] target, int k, Query filter) {
   public Query rewrite(IndexReader reader) throws IOException {
     TopDocs[] perLeafResults = new TopDocs[reader.leaves().size()];
 
-    BitSetCollector filterCollector = null;
+    Weight filterWeight = null;
     if (filter != null) {
-      filterCollector = new BitSetCollector(reader.leaves().size());
       IndexSearcher indexSearcher = new IndexSearcher(reader);
       BooleanQuery booleanQuery =
           new BooleanQuery.Builder()
               .add(filter, BooleanClause.Occur.FILTER)
               .add(new FieldExistsQuery(field), BooleanClause.Occur.FILTER)
               .build();
-      indexSearcher.search(booleanQuery, filterCollector);
+      Query rewritten = indexSearcher.rewrite(booleanQuery);
+      filterWeight = indexSearcher.createWeight(rewritten, ScoreMode.COMPLETE_NO_SCORES, 1f);
     }
 
     for (LeafReaderContext ctx : reader.leaves()) {
-      TopDocs results = searchLeaf(ctx, filterCollector);
+      Bits acceptDocs;
+      int cost;
+      if (filterWeight != null) {
+        Scorer scorer = filterWeight.scorer(ctx);
+        if (scorer != null) {
+          DocIdSetIterator iterator = scorer.iterator();
+          if (iterator instanceof BitSetIterator) {
+            acceptDocs = ((BitSetIterator) iterator).getBitSet();
+          } else {
+            acceptDocs = BitSet.of(iterator, ctx.reader().maxDoc());
+          }
+          cost = (int) iterator.cost();

Review Comment:
   You're right.. the `scorer` seems to be overestimating quite a lot! I changed it to `cardinality` of the `BitSet`, and it only adds a small latency
   
   However, as @jpountz pointed out, it does not include `liveDocs` yet
   We need some way of incorporating these `liveDocs` into our `BitSet` without iterating one-by-one over matching bits. Any suggestions for 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] kaivalnp commented on a diff in pull request #951: LUCENE-10606: Optimize Prefilter Hit Collection

Posted by GitBox <gi...@apache.org>.
kaivalnp commented on code in PR #951:
URL: https://github.com/apache/lucene/pull/951#discussion_r905065952


##########
lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java:
##########
@@ -92,20 +91,40 @@ public KnnVectorQuery(String field, float[] target, int k, Query filter) {
   public Query rewrite(IndexReader reader) throws IOException {
     TopDocs[] perLeafResults = new TopDocs[reader.leaves().size()];
 
-    BitSetCollector filterCollector = null;
+    Weight filterWeight = null;
     if (filter != null) {
-      filterCollector = new BitSetCollector(reader.leaves().size());
       IndexSearcher indexSearcher = new IndexSearcher(reader);
       BooleanQuery booleanQuery =
           new BooleanQuery.Builder()
               .add(filter, BooleanClause.Occur.FILTER)
               .add(new FieldExistsQuery(field), BooleanClause.Occur.FILTER)
               .build();
-      indexSearcher.search(booleanQuery, filterCollector);
+      Query rewritten = indexSearcher.rewrite(booleanQuery);
+      filterWeight = indexSearcher.createWeight(rewritten, ScoreMode.COMPLETE_NO_SCORES, 1f);
     }
 
     for (LeafReaderContext ctx : reader.leaves()) {
-      TopDocs results = searchLeaf(ctx, filterCollector);
+      Bits acceptDocs;
+      int cost;
+      if (filterWeight != null) {
+        Scorer scorer = filterWeight.scorer(ctx);
+        if (scorer != null) {
+          DocIdSetIterator iterator = scorer.iterator();
+          if (iterator instanceof BitSetIterator) {
+            acceptDocs = ((BitSetIterator) iterator).getBitSet();
+          } else {
+            acceptDocs = BitSet.of(iterator, ctx.reader().maxDoc());
+          }
+          cost = (int) iterator.cost();

Review Comment:
   The approach you mentioned makes sense, and I have added it into the latest commit.
   
   But I feel we are missing out on some performance because of inaccurate estimation for `approximateSearch`. We could try @jpountz's suggestion of pro-rating the cost of the `BitSet` (since we only want some estimation), and performing `approximateSearch` on that. Something like:
   
   ```java
   Bits acceptDocs, liveDocs = ctx.reader().getLiveDocs();
   int maxDoc = ctx.reader().maxDoc(), cost;
   DocIdSetIterator iterator = scorer.iterator();
   if (iterator instanceof BitSetIterator bitSetIterator) {
     BitSet bitSet = bitSetIterator.getBitSet();
     acceptDocs = new Bits() {
       @Override
       public boolean get(int index) {
         return bitSet.get(index) && (liveDocs == null || liveDocs.get(index));
       }
   
       @Override
       public int length() {
         return maxDoc;
       }
     };
     cost = bitSet.cardinality() * ctx.reader().numDocs() / maxDoc;
     TopDocs results = approximateSearch(ctx, acceptDocs, cost);
     if (results.totalHits.relation == TotalHits.Relation.EQUAL_TO) {
       return results;
     } else {
       FilteredDocIdSetIterator filterIterator = new FilteredDocIdSetIterator(iterator) {
         @Override
         protected boolean match(int doc) {
           return liveDocs == null || liveDocs.get(doc);
         }
       };
       return exactSearch(ctx, filterIterator);
     }
   }
   ```
   
   This way we won't traverse the `BitSet` even for deletions. If the limit is reached, we anyways have to iterate over the `iterator` to build a `BitSet`, so instead we can pass the filtered `iterator` to `exactSearch`



-- 
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 diff in pull request #951: LUCENE-10606: Optimize Prefilter Hit Collection

Posted by GitBox <gi...@apache.org>.
jtibshirani commented on code in PR #951:
URL: https://github.com/apache/lucene/pull/951#discussion_r904419498


##########
lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java:
##########
@@ -92,20 +91,40 @@ public KnnVectorQuery(String field, float[] target, int k, Query filter) {
   public Query rewrite(IndexReader reader) throws IOException {
     TopDocs[] perLeafResults = new TopDocs[reader.leaves().size()];
 
-    BitSetCollector filterCollector = null;
+    Weight filterWeight = null;
     if (filter != null) {
-      filterCollector = new BitSetCollector(reader.leaves().size());
       IndexSearcher indexSearcher = new IndexSearcher(reader);
       BooleanQuery booleanQuery =
           new BooleanQuery.Builder()
               .add(filter, BooleanClause.Occur.FILTER)
               .add(new FieldExistsQuery(field), BooleanClause.Occur.FILTER)
               .build();
-      indexSearcher.search(booleanQuery, filterCollector);
+      Query rewritten = indexSearcher.rewrite(booleanQuery);
+      filterWeight = indexSearcher.createWeight(rewritten, ScoreMode.COMPLETE_NO_SCORES, 1f);
     }
 
     for (LeafReaderContext ctx : reader.leaves()) {
-      TopDocs results = searchLeaf(ctx, filterCollector);
+      Bits acceptDocs;
+      int cost;
+      if (filterWeight != null) {
+        Scorer scorer = filterWeight.scorer(ctx);
+        if (scorer != null) {
+          DocIdSetIterator iterator = scorer.iterator();
+          if (iterator instanceof BitSetIterator) {
+            acceptDocs = ((BitSetIterator) iterator).getBitSet();
+          } else {
+            acceptDocs = BitSet.of(iterator, ctx.reader().maxDoc());
+          }
+          cost = (int) iterator.cost();

Review Comment:
   I don't see a good way to do this, since `liveDocs` is not backed by a `FixedBitSet` 🤔 I left a general comment below on our next steps.



-- 
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] kaivalnp commented on a diff in pull request #951: LUCENE-10606: Optimize Prefilter Hit Collection

Posted by GitBox <gi...@apache.org>.
kaivalnp commented on code in PR #951:
URL: https://github.com/apache/lucene/pull/951#discussion_r908867024


##########
lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java:
##########
@@ -92,20 +91,40 @@ public KnnVectorQuery(String field, float[] target, int k, Query filter) {
   public Query rewrite(IndexReader reader) throws IOException {
     TopDocs[] perLeafResults = new TopDocs[reader.leaves().size()];
 
-    BitSetCollector filterCollector = null;
+    Weight filterWeight = null;
     if (filter != null) {
-      filterCollector = new BitSetCollector(reader.leaves().size());
       IndexSearcher indexSearcher = new IndexSearcher(reader);
       BooleanQuery booleanQuery =
           new BooleanQuery.Builder()
               .add(filter, BooleanClause.Occur.FILTER)
               .add(new FieldExistsQuery(field), BooleanClause.Occur.FILTER)
               .build();
-      indexSearcher.search(booleanQuery, filterCollector);
+      Query rewritten = indexSearcher.rewrite(booleanQuery);
+      filterWeight = indexSearcher.createWeight(rewritten, ScoreMode.COMPLETE_NO_SCORES, 1f);
     }
 
     for (LeafReaderContext ctx : reader.leaves()) {
-      TopDocs results = searchLeaf(ctx, filterCollector);
+      Bits acceptDocs;
+      int cost;
+      if (filterWeight != null) {
+        Scorer scorer = filterWeight.scorer(ctx);
+        if (scorer != null) {
+          DocIdSetIterator iterator = scorer.iterator();
+          if (iterator instanceof BitSetIterator) {
+            acceptDocs = ((BitSetIterator) iterator).getBitSet();
+          } else {
+            acceptDocs = BitSet.of(iterator, ctx.reader().maxDoc());
+          }
+          cost = (int) iterator.cost();

Review Comment:
   Yes, I think we should add another case where if both are `FixedBitSet`s, we can do a `FixedBitSet.and` to compute live + matching docs.. It might make the optimization more common



-- 
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] kaivalnp commented on pull request #951: LUCENE-10606: Optimize Prefilter Hit Collection

Posted by GitBox <gi...@apache.org>.
kaivalnp commented on PR #951:
URL: https://github.com/apache/lucene/pull/951#issuecomment-1166057877

   As for the test, I saw that many existing ones use `BitSet` internally (for example `testRandomWithFilter`) since it first caches into `LRUQueryCache` and the `scorer` produces a `BitSetIterator`. Is it okay to rely on these existing tests?


-- 
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 diff in pull request #951: LUCENE-10606: Optimize Prefilter Hit Collection

Posted by GitBox <gi...@apache.org>.
jtibshirani commented on code in PR #951:
URL: https://github.com/apache/lucene/pull/951#discussion_r905236130


##########
lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java:
##########
@@ -92,20 +91,40 @@ public KnnVectorQuery(String field, float[] target, int k, Query filter) {
   public Query rewrite(IndexReader reader) throws IOException {
     TopDocs[] perLeafResults = new TopDocs[reader.leaves().size()];
 
-    BitSetCollector filterCollector = null;
+    Weight filterWeight = null;
     if (filter != null) {
-      filterCollector = new BitSetCollector(reader.leaves().size());
       IndexSearcher indexSearcher = new IndexSearcher(reader);
       BooleanQuery booleanQuery =
           new BooleanQuery.Builder()
               .add(filter, BooleanClause.Occur.FILTER)
               .add(new FieldExistsQuery(field), BooleanClause.Occur.FILTER)
               .build();
-      indexSearcher.search(booleanQuery, filterCollector);
+      Query rewritten = indexSearcher.rewrite(booleanQuery);
+      filterWeight = indexSearcher.createWeight(rewritten, ScoreMode.COMPLETE_NO_SCORES, 1f);
     }
 
     for (LeafReaderContext ctx : reader.leaves()) {
-      TopDocs results = searchLeaf(ctx, filterCollector);
+      Bits acceptDocs;
+      int cost;
+      if (filterWeight != null) {
+        Scorer scorer = filterWeight.scorer(ctx);
+        if (scorer != null) {
+          DocIdSetIterator iterator = scorer.iterator();
+          if (iterator instanceof BitSetIterator) {
+            acceptDocs = ((BitSetIterator) iterator).getBitSet();
+          } else {
+            acceptDocs = BitSet.of(iterator, ctx.reader().maxDoc());
+          }
+          cost = (int) iterator.cost();

Review Comment:
   Personally I'd prefer to keep `visitedLimit` always an accurate representation. I think it makes the algorithm easier to reason about. We can always revisit this in the future with more ideas for optimizations.



-- 
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 diff in pull request #951: LUCENE-10606: Optimize Prefilter Hit Collection

Posted by GitBox <gi...@apache.org>.
jpountz commented on code in PR #951:
URL: https://github.com/apache/lucene/pull/951#discussion_r903360932


##########
lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java:
##########
@@ -92,20 +91,40 @@ public KnnVectorQuery(String field, float[] target, int k, Query filter) {
   public Query rewrite(IndexReader reader) throws IOException {
     TopDocs[] perLeafResults = new TopDocs[reader.leaves().size()];
 
-    BitSetCollector filterCollector = null;
+    Weight filterWeight = null;
     if (filter != null) {
-      filterCollector = new BitSetCollector(reader.leaves().size());
       IndexSearcher indexSearcher = new IndexSearcher(reader);
       BooleanQuery booleanQuery =
           new BooleanQuery.Builder()
               .add(filter, BooleanClause.Occur.FILTER)
               .add(new FieldExistsQuery(field), BooleanClause.Occur.FILTER)
               .build();
-      indexSearcher.search(booleanQuery, filterCollector);
+      Query rewritten = indexSearcher.rewrite(booleanQuery);
+      filterWeight = indexSearcher.createWeight(rewritten, ScoreMode.COMPLETE_NO_SCORES, 1f);
     }
 
     for (LeafReaderContext ctx : reader.leaves()) {
-      TopDocs results = searchLeaf(ctx, filterCollector);
+      Bits acceptDocs;
+      int cost;
+      if (filterWeight != null) {
+        Scorer scorer = filterWeight.scorer(ctx);
+        if (scorer != null) {
+          DocIdSetIterator iterator = scorer.iterator();
+          if (iterator instanceof BitSetIterator) {
+            acceptDocs = ((BitSetIterator) iterator).getBitSet();
+          } else {
+            acceptDocs = BitSet.of(iterator, ctx.reader().maxDoc());
+          }

Review Comment:
   > We can extend the BitSetIterator so that it also incorporates liveDocs (return the nextSetBit only if it is live, else move to the next bit in loop)
   
   FYI we have `FilteredDocIdSetIterator` which makes it easy to do something like this easily.
   
   > But we can't find an accurate estimate of the number of matching + live docs (as it is needed in visitedLimit to switch over to exactSearch)?
   
   No indeed. We'll need to use an approximate value when there are deletes if we want to be able to benefit from the speedup of this change. I guess this means either ignoring deletes or pro-rating the number of bits that are set in the bitset by the ratio of live docs.



-- 
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 diff in pull request #951: LUCENE-10606: Optimize Prefilter Hit Collection

Posted by GitBox <gi...@apache.org>.
msokolov commented on code in PR #951:
URL: https://github.com/apache/lucene/pull/951#discussion_r908461540


##########
lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java:
##########
@@ -92,20 +91,40 @@ public KnnVectorQuery(String field, float[] target, int k, Query filter) {
   public Query rewrite(IndexReader reader) throws IOException {
     TopDocs[] perLeafResults = new TopDocs[reader.leaves().size()];
 
-    BitSetCollector filterCollector = null;
+    Weight filterWeight = null;
     if (filter != null) {
-      filterCollector = new BitSetCollector(reader.leaves().size());
       IndexSearcher indexSearcher = new IndexSearcher(reader);
       BooleanQuery booleanQuery =
           new BooleanQuery.Builder()
               .add(filter, BooleanClause.Occur.FILTER)
               .add(new FieldExistsQuery(field), BooleanClause.Occur.FILTER)
               .build();
-      indexSearcher.search(booleanQuery, filterCollector);
+      Query rewritten = indexSearcher.rewrite(booleanQuery);
+      filterWeight = indexSearcher.createWeight(rewritten, ScoreMode.COMPLETE_NO_SCORES, 1f);
     }
 
     for (LeafReaderContext ctx : reader.leaves()) {
-      TopDocs results = searchLeaf(ctx, filterCollector);
+      Bits acceptDocs;
+      int cost;
+      if (filterWeight != null) {
+        Scorer scorer = filterWeight.scorer(ctx);
+        if (scorer != null) {
+          DocIdSetIterator iterator = scorer.iterator();
+          if (iterator instanceof BitSetIterator) {
+            acceptDocs = ((BitSetIterator) iterator).getBitSet();
+          } else {
+            acceptDocs = BitSet.of(iterator, ctx.reader().maxDoc());
+          }
+          cost = (int) iterator.cost();

Review Comment:
   Could we apply the optimization using liveDocs in the case that it *is* a FixedBitSet? can we tell?



-- 
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 diff in pull request #951: LUCENE-10606: Optimize Prefilter Hit Collection

Posted by GitBox <gi...@apache.org>.
jpountz commented on code in PR #951:
URL: https://github.com/apache/lucene/pull/951#discussion_r907105526


##########
lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java:
##########
@@ -92,20 +91,40 @@ public KnnVectorQuery(String field, float[] target, int k, Query filter) {
   public Query rewrite(IndexReader reader) throws IOException {
     TopDocs[] perLeafResults = new TopDocs[reader.leaves().size()];
 
-    BitSetCollector filterCollector = null;
+    Weight filterWeight = null;
     if (filter != null) {
-      filterCollector = new BitSetCollector(reader.leaves().size());
       IndexSearcher indexSearcher = new IndexSearcher(reader);
       BooleanQuery booleanQuery =
           new BooleanQuery.Builder()
               .add(filter, BooleanClause.Occur.FILTER)
               .add(new FieldExistsQuery(field), BooleanClause.Occur.FILTER)
               .build();
-      indexSearcher.search(booleanQuery, filterCollector);
+      Query rewritten = indexSearcher.rewrite(booleanQuery);
+      filterWeight = indexSearcher.createWeight(rewritten, ScoreMode.COMPLETE_NO_SCORES, 1f);
     }
 
     for (LeafReaderContext ctx : reader.leaves()) {
-      TopDocs results = searchLeaf(ctx, filterCollector);
+      Bits acceptDocs;
+      int cost;
+      if (filterWeight != null) {
+        Scorer scorer = filterWeight.scorer(ctx);
+        if (scorer != null) {
+          DocIdSetIterator iterator = scorer.iterator();
+          if (iterator instanceof BitSetIterator) {
+            acceptDocs = ((BitSetIterator) iterator).getBitSet();
+          } else {
+            acceptDocs = BitSet.of(iterator, ctx.reader().maxDoc());
+          }
+          cost = (int) iterator.cost();

Review Comment:
   > I don't see a good way to do this, since liveDocs is not backed by a FixedBitSet
   
   FWIW there is no guarantee that liveDocs are backed by a FixedBitSet, but the default codec always uses a FixedBitSet.



-- 
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 diff in pull request #951: LUCENE-10606: Optimize Prefilter Hit Collection

Posted by GitBox <gi...@apache.org>.
jpountz commented on code in PR #951:
URL: https://github.com/apache/lucene/pull/951#discussion_r902383800


##########
lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java:
##########
@@ -92,20 +91,40 @@ public KnnVectorQuery(String field, float[] target, int k, Query filter) {
   public Query rewrite(IndexReader reader) throws IOException {
     TopDocs[] perLeafResults = new TopDocs[reader.leaves().size()];
 
-    BitSetCollector filterCollector = null;
+    Weight filterWeight = null;
     if (filter != null) {
-      filterCollector = new BitSetCollector(reader.leaves().size());
       IndexSearcher indexSearcher = new IndexSearcher(reader);
       BooleanQuery booleanQuery =
           new BooleanQuery.Builder()
               .add(filter, BooleanClause.Occur.FILTER)
               .add(new FieldExistsQuery(field), BooleanClause.Occur.FILTER)
               .build();
-      indexSearcher.search(booleanQuery, filterCollector);
+      Query rewritten = indexSearcher.rewrite(booleanQuery);
+      filterWeight = indexSearcher.createWeight(rewritten, ScoreMode.COMPLETE_NO_SCORES, 1f);
     }
 
     for (LeafReaderContext ctx : reader.leaves()) {
-      TopDocs results = searchLeaf(ctx, filterCollector);
+      Bits acceptDocs;
+      int cost;
+      if (filterWeight != null) {
+        Scorer scorer = filterWeight.scorer(ctx);
+        if (scorer != null) {
+          DocIdSetIterator iterator = scorer.iterator();
+          if (iterator instanceof BitSetIterator) {
+            acceptDocs = ((BitSetIterator) iterator).getBitSet();
+          } else {
+            acceptDocs = BitSet.of(iterator, ctx.reader().maxDoc());
+          }

Review Comment:
   Is it a problem? `exactSearch` doesn't need a `BitSet` but a `DocIdSetIterator`, which should be easy to create by filtering the scorer's iterator to exclude live docs?



-- 
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] kaivalnp commented on a diff in pull request #951: LUCENE-10606: Optimize Prefilter Hit Collection

Posted by GitBox <gi...@apache.org>.
kaivalnp commented on code in PR #951:
URL: https://github.com/apache/lucene/pull/951#discussion_r906442078


##########
lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java:
##########
@@ -121,36 +120,50 @@ public Query rewrite(IndexReader reader) throws IOException {
     return createRewrittenQuery(reader, topK);
   }
 
-  private TopDocs searchLeaf(LeafReaderContext ctx, BitSetCollector filterCollector)
-      throws IOException {
+  private TopDocs searchLeaf(LeafReaderContext ctx, Weight filterWeight) throws IOException {
+    Bits liveDocs = ctx.reader().getLiveDocs();
+    int maxDoc = ctx.reader().maxDoc();
 
-    if (filterCollector == null) {
-      Bits acceptDocs = ctx.reader().getLiveDocs();
-      return approximateSearch(ctx, acceptDocs, Integer.MAX_VALUE);
+    if (filterWeight == null) {
+      return approximateSearch(ctx, liveDocs, Integer.MAX_VALUE);
     } else {
-      BitSetIterator filterIterator = filterCollector.getIterator(ctx.ord);
-      if (filterIterator == null || filterIterator.cost() == 0) {
+      Scorer scorer = filterWeight.scorer(ctx);
+      if (scorer == null) {
         return NO_RESULTS;
-      }
+      } else {
+        BitSetIterator filterIterator =
+            cacheIntoBitSetIterator(scorer.iterator(), liveDocs, maxDoc);
 
-      if (filterIterator.cost() <= k) {
-        // If there are <= k possible matches, short-circuit and perform exact search, since HNSW
-        // must always visit at least k documents
-        return exactSearch(ctx, filterIterator);
+        if (filterIterator.cost() <= k) {
+          return exactSearch(ctx, filterIterator);
+        }
+        TopDocs results =
+            approximateSearch(ctx, filterIterator.getBitSet(), (int) filterIterator.cost());
+        if (results.totalHits.relation == TotalHits.Relation.EQUAL_TO) {
+          return results;
+        } else {
+          return exactSearch(ctx, filterIterator);
+        }
       }
+    }
+  }
 
-      // Perform the approximate kNN search
-      Bits acceptDocs =
-          filterIterator.getBitSet(); // The filter iterator already incorporates live docs
-      int visitedLimit = (int) filterIterator.cost();
-      TopDocs results = approximateSearch(ctx, acceptDocs, visitedLimit);
-      if (results.totalHits.relation == TotalHits.Relation.EQUAL_TO) {
-        return results;
-      } else {
-        // We stopped the kNN search because it visited too many nodes, so fall back to exact search
-        return exactSearch(ctx, filterIterator);
-      }
+  private BitSetIterator cacheIntoBitSetIterator(

Review Comment:
   Sure, we can return a `BitSet` for clarity..
   I think using `cost` there was justified since it was calculated and set during hit collection itself (and was accurate)



-- 
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 diff in pull request #951: LUCENE-10606: Optimize Prefilter Hit Collection

Posted by GitBox <gi...@apache.org>.
jtibshirani commented on code in PR #951:
URL: https://github.com/apache/lucene/pull/951#discussion_r896012687


##########
lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java:
##########
@@ -92,20 +91,40 @@ public KnnVectorQuery(String field, float[] target, int k, Query filter) {
   public Query rewrite(IndexReader reader) throws IOException {
     TopDocs[] perLeafResults = new TopDocs[reader.leaves().size()];
 
-    BitSetCollector filterCollector = null;
+    Weight filterWeight = null;
     if (filter != null) {
-      filterCollector = new BitSetCollector(reader.leaves().size());
       IndexSearcher indexSearcher = new IndexSearcher(reader);
       BooleanQuery booleanQuery =
           new BooleanQuery.Builder()
               .add(filter, BooleanClause.Occur.FILTER)
               .add(new FieldExistsQuery(field), BooleanClause.Occur.FILTER)
               .build();
-      indexSearcher.search(booleanQuery, filterCollector);
+      Query rewritten = indexSearcher.rewrite(booleanQuery);
+      filterWeight = indexSearcher.createWeight(rewritten, ScoreMode.COMPLETE_NO_SCORES, 1f);
     }
 
     for (LeafReaderContext ctx : reader.leaves()) {
-      TopDocs results = searchLeaf(ctx, filterCollector);
+      Bits acceptDocs;
+      int cost;
+      if (filterWeight != null) {
+        Scorer scorer = filterWeight.scorer(ctx);
+        if (scorer != null) {
+          DocIdSetIterator iterator = scorer.iterator();
+          if (iterator instanceof BitSetIterator) {
+            acceptDocs = ((BitSetIterator) iterator).getBitSet();
+          } else {
+            acceptDocs = BitSet.of(iterator, ctx.reader().maxDoc());
+          }
+          cost = (int) iterator.cost();

Review Comment:
   This changes the meaning of `cost` (which is directly used as `visitedLimit`). Before we were using the exact number of matches, whereas now we ask the iterator for a cost estimation. These cost estimates are sometimes very imprecise, and I worry it could make the query performance unpredictable and harder to understand.
   
   It wonder if we could convert everything to a `BitSet` and then use the actual cardinality. Hopefully we could do this while still keeping the nice performance improvement?



##########
lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java:
##########
@@ -121,35 +140,15 @@ public Query rewrite(IndexReader reader) throws IOException {
     return createRewrittenQuery(reader, topK);
   }
 
-  private TopDocs searchLeaf(LeafReaderContext ctx, BitSetCollector filterCollector)
-      throws IOException {
-
-    if (filterCollector == null) {
-      Bits acceptDocs = ctx.reader().getLiveDocs();
-      return approximateSearch(ctx, acceptDocs, Integer.MAX_VALUE);
+  private TopDocs searchLeaf(LeafReaderContext ctx, Bits acceptDocs, int cost) throws IOException {
+    TopDocs results = approximateSearch(ctx, acceptDocs, cost);

Review Comment:
   The new logic here drops this check -- could we make sure to keep it?
   ```
         if (filterIterator.cost() <= k) {
           // If there are <= k possible matches, short-circuit and perform exact search, since HNSW
           // must always visit at least k documents
           return exactSearch(ctx, filterIterator);
         }
   ```



-- 
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 #951: LUCENE-10606: Optimize Prefilter Hit Collection

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

   @kaivalnp just wanted to check how this is going. I'm excited about this improvement. Let me know if I can help with anything, for example I could dig into the questions that Adrien and I raised earlier.


-- 
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] kaivalnp commented on pull request #951: LUCENE-10606: Optimize Prefilter Hit Collection

Posted by GitBox <gi...@apache.org>.
kaivalnp commented on PR #951:
URL: https://github.com/apache/lucene/pull/951#issuecomment-1164455602

   Thank you! I have added this approach to the latest commit, and a suggestion to incorporate deletes above


-- 
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 #951: LUCENE-10606: Optimize Prefilter Hit Collection

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

   The latest approach looks good to me. Are you still seeing a significant latency improvement in some cases?


-- 
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 diff in pull request #951: LUCENE-10606: Optimize Prefilter Hit Collection

Posted by GitBox <gi...@apache.org>.
jtibshirani commented on code in PR #951:
URL: https://github.com/apache/lucene/pull/951#discussion_r906418634


##########
lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java:
##########
@@ -121,36 +120,50 @@ public Query rewrite(IndexReader reader) throws IOException {
     return createRewrittenQuery(reader, topK);
   }
 
-  private TopDocs searchLeaf(LeafReaderContext ctx, BitSetCollector filterCollector)
-      throws IOException {
+  private TopDocs searchLeaf(LeafReaderContext ctx, Weight filterWeight) throws IOException {
+    Bits liveDocs = ctx.reader().getLiveDocs();
+    int maxDoc = ctx.reader().maxDoc();
 
-    if (filterCollector == null) {
-      Bits acceptDocs = ctx.reader().getLiveDocs();
-      return approximateSearch(ctx, acceptDocs, Integer.MAX_VALUE);
+    if (filterWeight == null) {
+      return approximateSearch(ctx, liveDocs, Integer.MAX_VALUE);
     } else {
-      BitSetIterator filterIterator = filterCollector.getIterator(ctx.ord);
-      if (filterIterator == null || filterIterator.cost() == 0) {
+      Scorer scorer = filterWeight.scorer(ctx);
+      if (scorer == null) {
         return NO_RESULTS;
-      }
+      } else {
+        BitSetIterator filterIterator =
+            cacheIntoBitSetIterator(scorer.iterator(), liveDocs, maxDoc);
 
-      if (filterIterator.cost() <= k) {
-        // If there are <= k possible matches, short-circuit and perform exact search, since HNSW
-        // must always visit at least k documents
-        return exactSearch(ctx, filterIterator);
+        if (filterIterator.cost() <= k) {
+          return exactSearch(ctx, filterIterator);

Review Comment:
   Could we restore all the comments in this section? I think they're helpful in understanding the algorithm.



##########
lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java:
##########
@@ -121,36 +120,50 @@ public Query rewrite(IndexReader reader) throws IOException {
     return createRewrittenQuery(reader, topK);
   }
 
-  private TopDocs searchLeaf(LeafReaderContext ctx, BitSetCollector filterCollector)
-      throws IOException {
+  private TopDocs searchLeaf(LeafReaderContext ctx, Weight filterWeight) throws IOException {
+    Bits liveDocs = ctx.reader().getLiveDocs();
+    int maxDoc = ctx.reader().maxDoc();
 
-    if (filterCollector == null) {
-      Bits acceptDocs = ctx.reader().getLiveDocs();
-      return approximateSearch(ctx, acceptDocs, Integer.MAX_VALUE);
+    if (filterWeight == null) {
+      return approximateSearch(ctx, liveDocs, Integer.MAX_VALUE);
     } else {
-      BitSetIterator filterIterator = filterCollector.getIterator(ctx.ord);
-      if (filterIterator == null || filterIterator.cost() == 0) {
+      Scorer scorer = filterWeight.scorer(ctx);

Review Comment:
   Small suggestion, I often like to remove the "else" when the "if" statement has already returned a value. This avoids having a lot of highly nested else/ if statements. This suggestion applies to a few places in this method.



##########
lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java:
##########
@@ -121,36 +120,50 @@ public Query rewrite(IndexReader reader) throws IOException {
     return createRewrittenQuery(reader, topK);
   }
 
-  private TopDocs searchLeaf(LeafReaderContext ctx, BitSetCollector filterCollector)
-      throws IOException {
+  private TopDocs searchLeaf(LeafReaderContext ctx, Weight filterWeight) throws IOException {
+    Bits liveDocs = ctx.reader().getLiveDocs();
+    int maxDoc = ctx.reader().maxDoc();
 
-    if (filterCollector == null) {
-      Bits acceptDocs = ctx.reader().getLiveDocs();
-      return approximateSearch(ctx, acceptDocs, Integer.MAX_VALUE);
+    if (filterWeight == null) {
+      return approximateSearch(ctx, liveDocs, Integer.MAX_VALUE);
     } else {
-      BitSetIterator filterIterator = filterCollector.getIterator(ctx.ord);
-      if (filterIterator == null || filterIterator.cost() == 0) {
+      Scorer scorer = filterWeight.scorer(ctx);
+      if (scorer == null) {
         return NO_RESULTS;
-      }
+      } else {
+        BitSetIterator filterIterator =
+            cacheIntoBitSetIterator(scorer.iterator(), liveDocs, maxDoc);
 
-      if (filterIterator.cost() <= k) {
-        // If there are <= k possible matches, short-circuit and perform exact search, since HNSW
-        // must always visit at least k documents
-        return exactSearch(ctx, filterIterator);
+        if (filterIterator.cost() <= k) {
+          return exactSearch(ctx, filterIterator);
+        }
+        TopDocs results =
+            approximateSearch(ctx, filterIterator.getBitSet(), (int) filterIterator.cost());
+        if (results.totalHits.relation == TotalHits.Relation.EQUAL_TO) {
+          return results;
+        } else {
+          return exactSearch(ctx, filterIterator);
+        }
       }
+    }
+  }
 
-      // Perform the approximate kNN search
-      Bits acceptDocs =
-          filterIterator.getBitSet(); // The filter iterator already incorporates live docs
-      int visitedLimit = (int) filterIterator.cost();
-      TopDocs results = approximateSearch(ctx, acceptDocs, visitedLimit);
-      if (results.totalHits.relation == TotalHits.Relation.EQUAL_TO) {
-        return results;
-      } else {
-        // We stopped the kNN search because it visited too many nodes, so fall back to exact search
-        return exactSearch(ctx, filterIterator);
-      }
+  private BitSetIterator cacheIntoBitSetIterator(

Review Comment:
   Small comment, maybe it's clearer to return a `BitSet` here? Then we could just take the cardinality in the calling method. It was actually a little hacky before that we were using `BitSetIterator#cost` to always represent the cardinality (my fault!), when the cost is a separate concept from 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 pull request #951: LUCENE-10606: Optimize Prefilter Hit Collection

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

   I looked into this more deeply and realized that there are a bunch of times we decide not to cache a query into a `BitSet`. For example `UsageTrackingQueryCachingPolicy#shouldNeverCache` decides to never cache `TermQuery`. So with the default caching policy, the optimization may not kick in super often. At this point it's a loose best effort.
   
   For this reason, maybe we can keep the optimization simple for now. We could only apply it if the iterator is a `BitSetIterator` and there are no deleted docs. The conversion would look something like this:
   
   ```
   private BitSet createBitSet(Weight filterWeight, LeafReaderContext context) throws IOException {
           int maxDoc = context.reader().maxDoc();
           Bits liveDocs = context.reader().getLiveDocs();
   
           Scorer scorer = filterWeight.scorer(context);
           if (scorer == null) {
               return new FixedBitSet(maxDoc);
           }
   
           DocIdSetIterator iterator = scorer.iterator();
           if (liveDocs == null && iterator instanceof BitSetIterator bitSetIterator) {
               return bitSetIterator.getBitSet();
           }
   
           FixedBitSet bitSet = new FixedBitSet(maxDoc);
           for (int doc = iterator.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = iterator.nextDoc()) {
               if (liveDocs == null || liveDocs.get(doc)) {
                   bitSet.set(doc);
               }
           }
           return bitSet;
       }
   ```
   
   Then we would pass the `BitSet` for `acceptDocs`, and `BitSet#cardinality` for `visitedLimit`. That way the algorithm remains unchanged but we avoid creating a new bit set in some cases.
   
   I also think your approach in this PR makes the code cleaner, so that's a nice benefit.


-- 
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 diff in pull request #951: LUCENE-10606: Optimize Prefilter Hit Collection

Posted by GitBox <gi...@apache.org>.
jpountz commented on code in PR #951:
URL: https://github.com/apache/lucene/pull/951#discussion_r895488795


##########
lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java:
##########
@@ -92,20 +91,40 @@ public KnnVectorQuery(String field, float[] target, int k, Query filter) {
   public Query rewrite(IndexReader reader) throws IOException {
     TopDocs[] perLeafResults = new TopDocs[reader.leaves().size()];
 
-    BitSetCollector filterCollector = null;
+    Weight filterWeight = null;
     if (filter != null) {
-      filterCollector = new BitSetCollector(reader.leaves().size());
       IndexSearcher indexSearcher = new IndexSearcher(reader);
       BooleanQuery booleanQuery =
           new BooleanQuery.Builder()
               .add(filter, BooleanClause.Occur.FILTER)
               .add(new FieldExistsQuery(field), BooleanClause.Occur.FILTER)
               .build();
-      indexSearcher.search(booleanQuery, filterCollector);
+      Query rewritten = indexSearcher.rewrite(booleanQuery);
+      filterWeight = indexSearcher.createWeight(rewritten, ScoreMode.COMPLETE_NO_SCORES, 1f);
     }
 
     for (LeafReaderContext ctx : reader.leaves()) {
-      TopDocs results = searchLeaf(ctx, filterCollector);
+      Bits acceptDocs;
+      int cost;
+      if (filterWeight != null) {
+        Scorer scorer = filterWeight.scorer(ctx);
+        if (scorer != null) {
+          DocIdSetIterator iterator = scorer.iterator();
+          if (iterator instanceof BitSetIterator) {
+            acceptDocs = ((BitSetIterator) iterator).getBitSet();
+          } else {
+            acceptDocs = BitSet.of(iterator, ctx.reader().maxDoc());
+          }

Review Comment:
   Do we need to apply live docs here? `Scorer#iterator` returns an iterator over all matches, including deleted documents.



-- 
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] kaivalnp commented on pull request #951: LUCENE-10606: Optimize Prefilter Hit Collection

Posted by GitBox <gi...@apache.org>.
kaivalnp commented on PR #951:
URL: https://github.com/apache/lucene/pull/951#issuecomment-1164860104

   Yes, I saw similar improvement for `BitSet` backed queries as the numbers [here](https://github.com/apache/lucene/pull/932)


-- 
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] kaivalnp commented on a diff in pull request #951: LUCENE-10606: Optimize Prefilter Hit Collection

Posted by GitBox <gi...@apache.org>.
kaivalnp commented on code in PR #951:
URL: https://github.com/apache/lucene/pull/951#discussion_r903319874


##########
lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java:
##########
@@ -92,20 +91,40 @@ public KnnVectorQuery(String field, float[] target, int k, Query filter) {
   public Query rewrite(IndexReader reader) throws IOException {
     TopDocs[] perLeafResults = new TopDocs[reader.leaves().size()];
 
-    BitSetCollector filterCollector = null;
+    Weight filterWeight = null;
     if (filter != null) {
-      filterCollector = new BitSetCollector(reader.leaves().size());
       IndexSearcher indexSearcher = new IndexSearcher(reader);
       BooleanQuery booleanQuery =
           new BooleanQuery.Builder()
               .add(filter, BooleanClause.Occur.FILTER)
               .add(new FieldExistsQuery(field), BooleanClause.Occur.FILTER)
               .build();
-      indexSearcher.search(booleanQuery, filterCollector);
+      Query rewritten = indexSearcher.rewrite(booleanQuery);
+      filterWeight = indexSearcher.createWeight(rewritten, ScoreMode.COMPLETE_NO_SCORES, 1f);
     }
 
     for (LeafReaderContext ctx : reader.leaves()) {
-      TopDocs results = searchLeaf(ctx, filterCollector);
+      Bits acceptDocs;
+      int cost;
+      if (filterWeight != null) {
+        Scorer scorer = filterWeight.scorer(ctx);
+        if (scorer != null) {
+          DocIdSetIterator iterator = scorer.iterator();
+          if (iterator instanceof BitSetIterator) {
+            acceptDocs = ((BitSetIterator) iterator).getBitSet();
+          } else {
+            acceptDocs = BitSet.of(iterator, ctx.reader().maxDoc());
+          }

Review Comment:
   We can extend the `BitSetIterator` so that it also incorporates `liveDocs` (return the `nextSetBit` only if it is live, else move to the next bit in loop)
   But we can't find an accurate estimate of the number of matching + live docs (as it is needed in `visitedLimit` to switch over to `exactSearch`)?



-- 
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 #951: LUCENE-10606: Optimize Prefilter Hit Collection

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


-- 
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] kaivalnp commented on a diff in pull request #951: LUCENE-10606: Optimize Prefilter Hit Collection

Posted by GitBox <gi...@apache.org>.
kaivalnp commented on code in PR #951:
URL: https://github.com/apache/lucene/pull/951#discussion_r902336738


##########
lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java:
##########
@@ -121,35 +140,15 @@ public Query rewrite(IndexReader reader) throws IOException {
     return createRewrittenQuery(reader, topK);
   }
 
-  private TopDocs searchLeaf(LeafReaderContext ctx, BitSetCollector filterCollector)
-      throws IOException {
-
-    if (filterCollector == null) {
-      Bits acceptDocs = ctx.reader().getLiveDocs();
-      return approximateSearch(ctx, acceptDocs, Integer.MAX_VALUE);
+  private TopDocs searchLeaf(LeafReaderContext ctx, Bits acceptDocs, int cost) throws IOException {
+    TopDocs results = approximateSearch(ctx, acceptDocs, cost);

Review Comment:
   Yes, makes sense! Will add 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] kaivalnp commented on a diff in pull request #951: LUCENE-10606: Optimize Prefilter Hit Collection

Posted by GitBox <gi...@apache.org>.
kaivalnp commented on code in PR #951:
URL: https://github.com/apache/lucene/pull/951#discussion_r906440432


##########
lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java:
##########
@@ -121,36 +120,50 @@ public Query rewrite(IndexReader reader) throws IOException {
     return createRewrittenQuery(reader, topK);
   }
 
-  private TopDocs searchLeaf(LeafReaderContext ctx, BitSetCollector filterCollector)
-      throws IOException {
+  private TopDocs searchLeaf(LeafReaderContext ctx, Weight filterWeight) throws IOException {
+    Bits liveDocs = ctx.reader().getLiveDocs();
+    int maxDoc = ctx.reader().maxDoc();
 
-    if (filterCollector == null) {
-      Bits acceptDocs = ctx.reader().getLiveDocs();
-      return approximateSearch(ctx, acceptDocs, Integer.MAX_VALUE);
+    if (filterWeight == null) {
+      return approximateSearch(ctx, liveDocs, Integer.MAX_VALUE);
     } else {
-      BitSetIterator filterIterator = filterCollector.getIterator(ctx.ord);
-      if (filterIterator == null || filterIterator.cost() == 0) {
+      Scorer scorer = filterWeight.scorer(ctx);
+      if (scorer == null) {
         return NO_RESULTS;
-      }
+      } else {
+        BitSetIterator filterIterator =
+            cacheIntoBitSetIterator(scorer.iterator(), liveDocs, maxDoc);
 
-      if (filterIterator.cost() <= k) {
-        // If there are <= k possible matches, short-circuit and perform exact search, since HNSW
-        // must always visit at least k documents
-        return exactSearch(ctx, filterIterator);
+        if (filterIterator.cost() <= k) {
+          return exactSearch(ctx, filterIterator);

Review Comment:
   Sure.. Makes 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] kaivalnp commented on a diff in pull request #951: LUCENE-10606: Optimize Prefilter Hit Collection

Posted by GitBox <gi...@apache.org>.
kaivalnp commented on code in PR #951:
URL: https://github.com/apache/lucene/pull/951#discussion_r906440936


##########
lucene/core/src/java/org/apache/lucene/search/KnnVectorQuery.java:
##########
@@ -121,36 +120,50 @@ public Query rewrite(IndexReader reader) throws IOException {
     return createRewrittenQuery(reader, topK);
   }
 
-  private TopDocs searchLeaf(LeafReaderContext ctx, BitSetCollector filterCollector)
-      throws IOException {
+  private TopDocs searchLeaf(LeafReaderContext ctx, Weight filterWeight) throws IOException {
+    Bits liveDocs = ctx.reader().getLiveDocs();
+    int maxDoc = ctx.reader().maxDoc();
 
-    if (filterCollector == null) {
-      Bits acceptDocs = ctx.reader().getLiveDocs();
-      return approximateSearch(ctx, acceptDocs, Integer.MAX_VALUE);
+    if (filterWeight == null) {
+      return approximateSearch(ctx, liveDocs, Integer.MAX_VALUE);
     } else {
-      BitSetIterator filterIterator = filterCollector.getIterator(ctx.ord);
-      if (filterIterator == null || filterIterator.cost() == 0) {
+      Scorer scorer = filterWeight.scorer(ctx);

Review Comment:
   Thanks... Will do!



-- 
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 #951: LUCENE-10606: Optimize Prefilter Hit Collection

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

   For context, I also reran benchmarks and didn't see any slowdown to the typical case (not backed by a BitSet).


-- 
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