You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by "kaivalnp (via GitHub)" <gi...@apache.org> on 2023/10/14 12:39:09 UTC

[PR] Add support for radius-based vector searches [lucene]

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

   ### Description
   
   Background in #12579
   
   Add support for getting "all vectors within a radius" as opposed to getting the "topK closest vectors" in the current system
   
   ### Considerations
   
   I've tried to keep this change minimal and non-invasive by not modifying any APIs and re-using existing HNSW graphs -- changing the graph traversal and result collection criteria to:
   1. Visit all nodes (reachable from the entry node in the last level) that are within an outer "traversal" radius
   2. Collect all nodes that are within an inner "result" radius
   
   ### Advantages
   
   1. Queries that have a high number of "relevant" results will get all of those (not limited by `topK`)
   2. Conversely, arbitrary queries where many results are not "relevant" will not waste time in getting all `topK` (when some of them will be removed later)
   3. Results of HNSW searches need not be sorted - and we can store them in a plain list as opposed to min-max heaps (saving on `heapify` calls). Merging results from segments is also cheaper, where we just concatenate results as opposed to calculating the index-level `topK`
   
   On a higher level, finding `topK` results needed HNSW searches to happen in `#rewrite` because of an interdependence of results between segments - where we want to find the index-level `topK` from multiple segment-level results. This is kind of against Lucene's concept of segments being independently searchable sub-indexes?
   
   Moreover, we needed explicit concurrency (#12160) to perform these in parallel, and these shortcomings would be naturally overcome with the new objective of finding "all vectors within a radius" - inherently independent of results from another segment (so we can move searches to a more fitting place?)
   
   ### Caveats
   
   I could not find much precedent in using HNSW graphs this way (or even the radius-based search for that matter - please add links to existing work if someone is aware) and consequently marked all classes as `@lucene.experimental`
   
   For now I have re-used lots of functionality from `AbstractKnnVectorQuery` to keep this minimal, but if the use-case is accepted more widely we can look into writing more suitable queries (as mentioned above briefly)
   
   ### Next steps
   
   Run benchmarks with this new query to see how it compares to the `topK` based search


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


Re: [PR] Add support for similarity-based vector searches [lucene]

Posted by "kaivalnp (via GitHub)" <gi...@apache.org>.
kaivalnp commented on PR #12679:
URL: https://github.com/apache/lucene/pull/12679#issuecomment-1812941899

   > You still need to score the vectors to realize that they are in the iteration set or not
   
   Right, I meant that we need not score all *other* vectors to determine if the vector itself is a "hit" or not (we just need its similarity score to be above the `resultSimilarity`) - as opposed to KNN where it's not a simple "filter" like you mentioned
   
   > we do all this work in approximateSearch (because we need to score the values) only to throw it away
   
   I've tried to re-use some of this work to [directly reject](https://github.com/apache/lucene/blob/cad565439be512ac6e95a698007b1fc971173f00/lucene/core/src/java/org/apache/lucene/search/AbstractVectorSimilarityQuery.java#L119-L121) vectors that are above the `traversalSimilarity` but below the `resultSimilarity` (the ones that were [already scored from HNSW search](https://github.com/apache/lucene/blob/cad565439be512ac6e95a698007b1fc971173f00/lucene/core/src/java/org/apache/lucene/search/VectorSimilarityCollector.java#L66-L68)), without re-computing their scores
   
   I wonder if we can extend this further: [`visited`](https://github.com/apache/lucene/blob/main/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphSearcher.java#L236) marks all the nodes for which we have computed scores from HNSW search. However, anything that is "visited but not collected" will not make it to the final results. We can do this by passing the `visited` variable back to the `KnnCollector` by adding a new method like `setVisited(Bits)`?
   
   This is also usable in the current KNN-based search, wherever we fall back from `approximateSearch` to `exactSearch`. If the `KnnCollector` had information about whatever we have already scored in graph searches (but is not present in the results) -- we can prevent computing its similarity scores again from `exactSearch`, because we already know they are not present in the `topK`
   
   Right now we [score all vectors](https://github.com/apache/lucene/blob/main/lucene/core/src/java/org/apache/lucene/search/AbstractKnnVectorQuery.java#L177-L187) present in the `filter`, even if many of them are already scored and rejected in graph search
   
   [Here](https://github.com/apache/lucene/commit/2d6c0bfd4134b04c60be3864567211c824e7bc3c) are some very rough changes to support this -- what do you think @benwtrent?


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


Re: [PR] Add support for similarity-based vector searches [lucene]

Posted by "kaivalnp (via GitHub)" <gi...@apache.org>.
kaivalnp commented on code in PR #12679:
URL: https://github.com/apache/lucene/pull/12679#discussion_r1419660341


##########
lucene/CHANGES.txt:
##########
@@ -167,7 +167,10 @@ API Changes
 
 New Features
 ---------------------
-(No changes)
+
+* GITHUB#12679: Add support for similarity-based vector searches. Finds all vectors scoring above a `resultSimilarity`
+  while traversing the HNSW graph till better-scoring nodes are available, or the best candidate is below a score of
+  `traversalSimilarity` in the lowest level. (Aditya Prakash, Kaival Parikh)

Review Comment:
   Sorry, didn't get what you mean here?



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


Re: [PR] Add support for radius-based vector searches [lucene]

Posted by "shubhamvishu (via GitHub)" <gi...@apache.org>.
shubhamvishu commented on code in PR #12679:
URL: https://github.com/apache/lucene/pull/12679#discussion_r1359624510


##########
lucene/core/src/java/org/apache/lucene/search/AbstractRnnVectorQuery.java:
##########


Review Comment:
   Lets add some tests for these going forward?



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


Re: [PR] Add support for radius-based vector searches [lucene]

Posted by "kaivalnp (via GitHub)" <gi...@apache.org>.
kaivalnp commented on code in PR #12679:
URL: https://github.com/apache/lucene/pull/12679#discussion_r1362476143


##########
lucene/core/src/java/org/apache/lucene/search/RnnCollector.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A collector that performs radius-based vector searches. All vectors within an outer radius are
+ * traversed, and those within an inner radius are collected.
+ *
+ * @lucene.experimental
+ */
+public class RnnCollector extends AbstractKnnCollector {
+  private final float traversalThreshold, resultThreshold;
+  private final List<ScoreDoc> scoreDocList;
+
+  /**
+   * Performs radius-based vector searches.
+   *
+   * @param traversalThreshold similarity score corresponding to outer radius of graph traversal.
+   * @param resultThreshold similarity score corresponding to inner radius of result collection.
+   * @param visitLimit limit of graph nodes to visit.
+   */
+  public RnnCollector(float traversalThreshold, float resultThreshold, long visitLimit) {
+    super(Integer.MAX_VALUE, visitLimit);
+    assert traversalThreshold <= resultThreshold;
+    this.traversalThreshold = traversalThreshold;
+    this.resultThreshold = resultThreshold;
+    this.scoreDocList = new ArrayList<>();
+  }
+
+  @Override
+  public boolean collect(int docId, float similarity) {
+    if (similarity >= resultThreshold) {
+      return scoreDocList.add(new ScoreDoc(docId, similarity));
+    }
+    return false;
+  }
+
+  @Override
+  public float minCompetitiveSimilarity() {
+    return traversalThreshold;
+  }
+
+  @Override
+  // This does not return results in a sorted order to prevent unnecessary calculations (because we
+  // do not want to maintain the topK)

Review Comment:
   Moved to more suitable places



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


Re: [PR] Add support for radius-based vector searches [lucene]

Posted by "kaivalnp (via GitHub)" <gi...@apache.org>.
kaivalnp commented on PR #12679:
URL: https://github.com/apache/lucene/pull/12679#issuecomment-1766834111

   Thanks for the review @shubhamvishu! Addressed some of the comments above
   
   > Is it right to call it a radius-based search here?
   
   I think of it as finding all results within a high-dimensional circle / sphere / equivalent, and the radius-based search seems to capture the essence. Although "threshold-based search" may be more appropriate (since radius is tied to Euclidean Distance, and may not be easy to relate with Cosine Similarity or Dot Product)
   
   No strong opinions here, looking for others' thoughts as well on more appropriate naming..
   
   > The RnnFloatVectorQuery and RnnByteVectorQuery are almost the same
   
   The problem here is that we'll have to generalize many other (unrelated to this change) internal classes. I'll keep this to a separate issue


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


Re: [PR] Add support for radius-based vector searches [lucene]

Posted by "kaivalnp (via GitHub)" <gi...@apache.org>.
kaivalnp commented on PR #12679:
URL: https://github.com/apache/lucene/pull/12679#issuecomment-1766741617

   > If I read correctly, this query ends up calling LeafReader#searchNearestNeighbors with k=Integer.MAX_VALUE
   
   No, we're calling the [new API](https://github.com/apache/lucene/blob/main/lucene/core/src/java/org/apache/lucene/index/LeafReader.java#L329) (from [here](https://github.com/apache/lucene/blob/a384967b5ea631de940e327f4483888c80d09611/lucene/core/src/java/org/apache/lucene/search/RnnFloatVectorQuery.java#L57)) with a custom [`RnnCollector`](https://github.com/apache/lucene/blob/a384967b5ea631de940e327f4483888c80d09611/lucene/core/src/java/org/apache/lucene/search/RnnCollector.java#L28) that performs score-based HNSW searches (as opposed to the [old API](https://github.com/apache/lucene/blob/main/lucene/core/src/java/org/apache/lucene/index/LeafReader.java#L246-L247) that performs `topK`-based searches with `k=Integer.MAX_VALUE`)
   
   The `Integer.MAX_VALUE` passed [here](https://github.com/apache/lucene/blob/a384967b5ea631de940e327f4483888c80d09611/lucene/core/src/java/org/apache/lucene/search/AbstractRnnVectorQuery.java#L38) is just used in two places: [`#exactSearch`](https://github.com/apache/lucene/blob/a384967b5ea631de940e327f4483888c80d09611/lucene/core/src/java/org/apache/lucene/search/AbstractKnnVectorQuery.java#L174) (to instantiate a priority queue of size `k`) and [`#mergeLeafResults`](https://github.com/apache/lucene/blob/a384967b5ea631de940e327f4483888c80d09611/lucene/core/src/java/org/apache/lucene/search/AbstractKnnVectorQuery.java#L216) (to request for the best-scoring `k` hits across all segment results). We're overriding both functions in our implementation of `AbstractRnnVectorQuery` (because we do not want to limit to `topK` results)
   
   I think you're worried that we'll end up performing brute-force KNN on all documents in the segment, and *then* retain vectors above the threshold? What we instead aim to do is: starting from the entry node in the last level of HNSW graphs, we keep visiting candidates as long as they are above the `traversalThreshold`, all the while adding nodes above the `resultThreshold` as accepted results
   
   This is not necessarily slower than normal HNSW searches, provided the `traversalThreshold` is chosen suitably


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


Re: [PR] Add support for radius-based vector searches [lucene]

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on PR #12679:
URL: https://github.com/apache/lucene/pull/12679#issuecomment-1766983182

   > I think of it as finding all results within a high-dimensional circle / sphere / equivalent,
   
   dot-product, cosine, etc. don't really follow that same idea as you point out. I would prefer something like `VectorSimilarityQuery` or something.
   
   > E.g. could we abort the approximate search if the list maintained by the RnnCollector grows too large, and fall back to an exact search that is based on a TwoPhaseIterator instead of eagerly collecting all matches into a list?
   
   I agree with @jpountz concerns.
   
   The topDocs collector gets a replay of the matched documents. We should put sane limits here and prevent folks from getting 100,000s of matches (int & float value arrays) via approximate search. It seems like having a huge number like that could cause issues.
   


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


Re: [PR] Add support for similarity-based vector searches [lucene]

Posted by "kaivalnp (via GitHub)" <gi...@apache.org>.
kaivalnp commented on code in PR #12679:
URL: https://github.com/apache/lucene/pull/12679#discussion_r1411285972


##########
lucene/core/src/java/org/apache/lucene/search/AbstractVectorSimilarityQuery.java:
##########
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.Objects;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.util.Bits;
+
+/**
+ * Search for all (approximate) vectors above a similarity threshold.
+ *
+ * @lucene.experimental
+ */
+abstract class AbstractVectorSimilarityQuery extends Query {
+  protected final String field;
+  protected final float traversalSimilarity, resultSimilarity;
+  protected final long visitLimit;
+
+  /**
+   * Search for all (approximate) vectors above a similarity threshold. First performs a
+   * similarity-based graph search using {@link VectorSimilarityCollector} between {@link
+   * #traversalSimilarity} and {@link #resultSimilarity}. If this does not complete within a
+   * specified {@link #visitLimit}, returns a lazy-loading iterator over all vectors above the
+   * {@link #resultSimilarity}.
+   *
+   * @param field a field that has been indexed as a vector field.
+   * @param traversalSimilarity (lower) similarity score for graph traversal.
+   * @param resultSimilarity (higher) similarity score for result collection.
+   * @param visitLimit limit on number of nodes to visit before falling back to a lazy-loading
+   *     iterator.
+   */
+  AbstractVectorSimilarityQuery(
+      String field, float traversalSimilarity, float resultSimilarity, long visitLimit) {
+    this.field = Objects.requireNonNull(field, "field");
+    this.traversalSimilarity = traversalSimilarity;
+    this.resultSimilarity = resultSimilarity;
+    this.visitLimit = visitLimit;
+  }
+
+  abstract VectorScorer createVectorScorer(LeafReaderContext context) throws IOException;
+
+  protected abstract void approximateSearch(LeafReaderContext context, KnnCollector collector)
+      throws IOException;

Review Comment:
   > we do all this work in approximateSearch (because we need to score the values) only to throw it away
   
   Agreed, there is useful information in graph search that can be reused from exact search (also applicable for KNN queries)!
   
   I had opened #12820 to discuss some ideas



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


Re: [PR] Add support for similarity-based vector searches [lucene]

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12679:
URL: https://github.com/apache/lucene/pull/12679#discussion_r1419656899


##########
lucene/CHANGES.txt:
##########
@@ -167,7 +167,10 @@ API Changes
 
 New Features
 ---------------------
-(No changes)
+
+* GITHUB#12679: Add support for similarity-based vector searches. Finds all vectors scoring above a `resultSimilarity`
+  while traversing the HNSW graph till better-scoring nodes are available, or the best candidate is below a score of
+  `traversalSimilarity` in the lowest level. (Aditya Prakash, Kaival Parikh)

Review Comment:
   Maybe add the vector query names?



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


Re: [PR] Add support for radius-based vector searches [lucene]

Posted by "shubhamvishu (via GitHub)" <gi...@apache.org>.
shubhamvishu commented on code in PR #12679:
URL: https://github.com/apache/lucene/pull/12679#discussion_r1359606449


##########
lucene/core/src/java/org/apache/lucene/search/AbstractRnnVectorQuery.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.LeafReaderContext;
+
+/**
+ * Search for all (approximate) vectors within a radius using the {@link RnnCollector}.
+ *
+ * @lucene.experimental
+ */
+abstract class AbstractRnnVectorQuery extends AbstractKnnVectorQuery {
+  private static final TopDocs NO_RESULTS = TopDocsCollector.EMPTY_TOPDOCS;
+
+  protected final float traversalThreshold, resultThreshold;
+
+  public AbstractRnnVectorQuery(
+      String field, float traversalThreshold, float resultThreshold, Query filter) {
+    super(field, Integer.MAX_VALUE, filter);
+    assert traversalThreshold <= resultThreshold;

Review Comment:
   Lets just throw IAE here? Same for the `RnnCollector`.



##########
lucene/core/src/java/org/apache/lucene/search/AbstractRnnVectorQuery.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.LeafReaderContext;
+
+/**
+ * Search for all (approximate) vectors within a radius using the {@link RnnCollector}.
+ *
+ * @lucene.experimental
+ */
+abstract class AbstractRnnVectorQuery extends AbstractKnnVectorQuery {
+  private static final TopDocs NO_RESULTS = TopDocsCollector.EMPTY_TOPDOCS;
+
+  protected final float traversalThreshold, resultThreshold;

Review Comment:
   Maybe add some javadocs(just copy from Rnn[Byte/Float]VectorQuery cx) for what each of these would serve in this new query?



##########
lucene/core/src/java/org/apache/lucene/search/AbstractRnnVectorQuery.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.LeafReaderContext;
+
+/**
+ * Search for all (approximate) vectors within a radius using the {@link RnnCollector}.
+ *
+ * @lucene.experimental
+ */
+abstract class AbstractRnnVectorQuery extends AbstractKnnVectorQuery {
+  private static final TopDocs NO_RESULTS = TopDocsCollector.EMPTY_TOPDOCS;

Review Comment:
   I see its same as `AbstractKnnVectorQuery` but we could just statically import `TopDocsCollector.EMPTY_TOPDOCS` and use that instead?



##########
lucene/core/src/java/org/apache/lucene/search/AbstractRnnVectorQuery.java:
##########


Review Comment:
   Lets add some tests for these going forward



##########
lucene/core/src/java/org/apache/lucene/search/RnnCollector.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A collector that performs radius-based vector searches. All vectors within an outer radius are
+ * traversed, and those within an inner radius are collected.
+ *
+ * @lucene.experimental
+ */
+public class RnnCollector extends AbstractKnnCollector {
+  private final float traversalThreshold, resultThreshold;
+  private final List<ScoreDoc> scoreDocList;
+
+  /**
+   * Performs radius-based vector searches.
+   *
+   * @param traversalThreshold similarity score corresponding to outer radius of graph traversal.
+   * @param resultThreshold similarity score corresponding to inner radius of result collection.
+   * @param visitLimit limit of graph nodes to visit.
+   */
+  public RnnCollector(float traversalThreshold, float resultThreshold, long visitLimit) {
+    super(Integer.MAX_VALUE, visitLimit);
+    assert traversalThreshold <= resultThreshold;
+    this.traversalThreshold = traversalThreshold;
+    this.resultThreshold = resultThreshold;
+    this.scoreDocList = new ArrayList<>();
+  }
+
+  @Override
+  public boolean collect(int docId, float similarity) {
+    if (similarity >= resultThreshold) {
+      return scoreDocList.add(new ScoreDoc(docId, similarity));
+    }
+    return false;
+  }
+
+  @Override
+  public float minCompetitiveSimilarity() {
+    return traversalThreshold;
+  }
+
+  @Override
+  // This does not return results in a sorted order to prevent unnecessary calculations (because we
+  // do not want to maintain the topK)

Review Comment:
   Nit : Maybe move the javadoc above the annotation here and in other occurrences. Visually I just feel something off this way.



##########
lucene/core/src/java/org/apache/lucene/search/AbstractRnnVectorQuery.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.LeafReaderContext;
+
+/**
+ * Search for all (approximate) vectors within a radius using the {@link RnnCollector}.
+ *
+ * @lucene.experimental
+ */
+abstract class AbstractRnnVectorQuery extends AbstractKnnVectorQuery {
+  private static final TopDocs NO_RESULTS = TopDocsCollector.EMPTY_TOPDOCS;
+
+  protected final float traversalThreshold, resultThreshold;
+
+  public AbstractRnnVectorQuery(
+      String field, float traversalThreshold, float resultThreshold, Query filter) {
+    super(field, Integer.MAX_VALUE, filter);
+    assert traversalThreshold <= resultThreshold;
+    this.traversalThreshold = traversalThreshold;
+    this.resultThreshold = resultThreshold;
+  }
+
+  @Override
+  protected TopDocs exactSearch(LeafReaderContext context, DocIdSetIterator acceptIterator)
+      throws IOException {
+    @SuppressWarnings("resource")
+    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;
+    }
+
+    VectorScorer vectorScorer = createVectorScorer(context, fi);
+    List<ScoreDoc> scoreDocList = new ArrayList<>();
+
+    int doc;
+    while ((doc = acceptIterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+      boolean advanced = vectorScorer.advanceExact(doc);
+      assert advanced;
+
+      float score = vectorScorer.score();
+      if (score >= resultThreshold) {
+        scoreDocList.add(new ScoreDoc(doc, score));
+      }
+    }
+
+    TotalHits totalHits = new TotalHits(acceptIterator.cost(), TotalHits.Relation.EQUAL_TO);
+    return new TopDocs(totalHits, scoreDocList.toArray(ScoreDoc[]::new));
+  }
+
+  @Override
+  // Segment-level results are not sorted (because we do not want to maintain the topK), just
+  // concatenate them
+  protected TopDocs mergeLeafResults(TopDocs[] perLeafResults) {
+    long value = 0;
+    TotalHits.Relation relation = TotalHits.Relation.EQUAL_TO;
+    List<ScoreDoc> scoreDocList = new ArrayList<>();
+
+    for (TopDocs topDocs : perLeafResults) {
+      value += topDocs.totalHits.value;
+      if (topDocs.totalHits.relation == TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO) {
+        relation = TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO;
+      }
+      scoreDocList.addAll(List.of(topDocs.scoreDocs));
+    }
+
+    return new TopDocs(new TotalHits(value, relation), scoreDocList.toArray(ScoreDoc[]::new));
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    if (!super.equals(o)) return false;

Review Comment:
   Lets merge these two if statements? Same for below occurences



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


Re: [PR] Add support for radius-based vector searches [lucene]

Posted by "shubhamvishu (via GitHub)" <gi...@apache.org>.
shubhamvishu commented on PR #12679:
URL: https://github.com/apache/lucene/pull/12679#issuecomment-1763170985

   Thanks for adding this @kaivalnp! The idea makes sense to me, looking forward to the benchmarks results. I left some minor comments. Sharing some thoughts below :
   
   1. Is it right to call it a radius-based search here?. I understand we are calling the traversal threshold as the outer radius and the required threshold as inner radius but it doesn't sounds very intuitive to me in this context i.e. of graph. I would correlate radius more with the sort of edges of the graph or something similar but not the dot product score (atleast when visually forming a mind map)
   2. The `RnnFloatVectorQuery` and `RnnByteVectorQuery` are almost the same. Now I understand this has been existing and maybe a convention(?) to have separate implementations for byte and float vectors but this seems like a very good opportunity to make use of generics here and only have like `RnnVectorQuery`. I don't know if using generics here would make things more complex somehow or some unknowon caveats but to me that looks like a good approach to me here. Looking forward to everybody's thoughts on 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


Re: [PR] Add support for radius-based vector searches [lucene]

Posted by "jpountz (via GitHub)" <gi...@apache.org>.
jpountz commented on PR #12679:
URL: https://github.com/apache/lucene/pull/12679#issuecomment-1766795903

   Thanks for explaining, I had overlooked how the `Integer.MAX_VALUE` was used indeed. I'm still interested in figuring out if we can have stronger guarantees on the worst-case memory usage that this query could have (I believe noting prevents this list from growing unbounded? if the threshold is high?). E.g. could we abort the approximate search if the list maintained by the `RnnCollector` grows too large, and fall back to an exact search that is based on a `TwoPhaseIterator` instead of eagerly collecting all matches into a list?


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


Re: [PR] Add support for radius-based vector searches [lucene]

Posted by "kaivalnp (via GitHub)" <gi...@apache.org>.
kaivalnp commented on code in PR #12679:
URL: https://github.com/apache/lucene/pull/12679#discussion_r1362475149


##########
lucene/core/src/java/org/apache/lucene/search/AbstractRnnVectorQuery.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.LeafReaderContext;
+
+/**
+ * Search for all (approximate) vectors within a radius using the {@link RnnCollector}.
+ *
+ * @lucene.experimental
+ */
+abstract class AbstractRnnVectorQuery extends AbstractKnnVectorQuery {
+  private static final TopDocs NO_RESULTS = TopDocsCollector.EMPTY_TOPDOCS;
+
+  protected final float traversalThreshold, resultThreshold;
+
+  public AbstractRnnVectorQuery(
+      String field, float traversalThreshold, float resultThreshold, Query filter) {
+    super(field, Integer.MAX_VALUE, filter);
+    assert traversalThreshold <= resultThreshold;
+    this.traversalThreshold = traversalThreshold;
+    this.resultThreshold = resultThreshold;
+  }
+
+  @Override
+  protected TopDocs exactSearch(LeafReaderContext context, DocIdSetIterator acceptIterator)
+      throws IOException {
+    @SuppressWarnings("resource")
+    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;
+    }
+
+    VectorScorer vectorScorer = createVectorScorer(context, fi);
+    List<ScoreDoc> scoreDocList = new ArrayList<>();
+
+    int doc;
+    while ((doc = acceptIterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+      boolean advanced = vectorScorer.advanceExact(doc);
+      assert advanced;
+
+      float score = vectorScorer.score();
+      if (score >= resultThreshold) {
+        scoreDocList.add(new ScoreDoc(doc, score));
+      }
+    }
+
+    TotalHits totalHits = new TotalHits(acceptIterator.cost(), TotalHits.Relation.EQUAL_TO);
+    return new TopDocs(totalHits, scoreDocList.toArray(ScoreDoc[]::new));
+  }
+
+  @Override
+  // Segment-level results are not sorted (because we do not want to maintain the topK), just
+  // concatenate them
+  protected TopDocs mergeLeafResults(TopDocs[] perLeafResults) {
+    long value = 0;
+    TotalHits.Relation relation = TotalHits.Relation.EQUAL_TO;
+    List<ScoreDoc> scoreDocList = new ArrayList<>();
+
+    for (TopDocs topDocs : perLeafResults) {
+      value += topDocs.totalHits.value;
+      if (topDocs.totalHits.relation == TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO) {
+        relation = TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO;
+      }
+      scoreDocList.addAll(List.of(topDocs.scoreDocs));
+    }
+
+    return new TopDocs(new TotalHits(value, relation), scoreDocList.toArray(ScoreDoc[]::new));
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    if (!super.equals(o)) return false;

Review Comment:
   This was the default `equals` method generated by the IDE. Made it more in line with other classes now



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


Re: [PR] Add support for similarity-based vector searches [lucene]

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12679:
URL: https://github.com/apache/lucene/pull/12679#discussion_r1389741654


##########
lucene/core/src/java/org/apache/lucene/search/VectorSimilarityCollector.java:
##########
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import org.apache.lucene.util.BitSet;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.SparseFixedBitSet;
+
+/**
+ * Perform a similarity-based graph search.
+ *
+ * @lucene.experimental
+ */
+class VectorSimilarityCollector extends AbstractKnnCollector {
+  private static final Bits MATCH_ALL_BITS = new Bits.MatchAllBits(Integer.MAX_VALUE);
+
+  private final float traversalSimilarity, resultSimilarity;
+  private final List<ScoreDoc> scoreDocList;
+  private final BitSet visited;
+
+  /**
+   * Perform a similarity-based graph search. All nodes above a {@link #traversalSimilarity} are
+   * traversed, and all nodes above a {@link #resultSimilarity} are collected.
+   *
+   * @param traversalSimilarity (lower) similarity score for graph traversal.
+   * @param resultSimilarity (higher) similarity score for result collection.
+   * @param visitLimit limit on number of nodes to visit.
+   * @param maxDoc maximum docid of any node.
+   */
+  public VectorSimilarityCollector(
+      float traversalSimilarity, float resultSimilarity, long visitLimit, int maxDoc) {
+    super(1, visitLimit);
+    if (traversalSimilarity > resultSimilarity) {
+      throw new IllegalArgumentException("traversalSimilarity should be <= resultSimilarity");
+    }
+    this.traversalSimilarity = traversalSimilarity;
+    this.resultSimilarity = resultSimilarity;
+    this.scoreDocList = new ArrayList<>();
+
+    if (visitLimit == Long.MAX_VALUE) {
+      this.visited = null;
+    } else {
+      this.visited = new SparseFixedBitSet(maxDoc);
+    }

Review Comment:
   The graphSearcher already calls `incVisitedCount` as it occurs. I don't think you need to keep track of visited within this collector.



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


Re: [PR] Add support for radius-based vector searches [lucene]

Posted by "kaivalnp (via GitHub)" <gi...@apache.org>.
kaivalnp commented on PR #12679:
URL: https://github.com/apache/lucene/pull/12679#issuecomment-1767022898

   > stronger guarantees on the worst-case memory usage
   
   Totally agreed @jpountz! It is very easy to go wrong in the new API, specially if the user passes a low threshold (high radius -> low threshold). As we can see from benchmarks above, the number of nodes to visit may jump very fast with slight reduction in the `traversalThreshold` (`mean` column of first table)
   
   > fall back to an exact search that is based on a `TwoPhaseIterator`
   
   This makes sense to me.. Something like a lazy-loading iterator, where we perform vector comparisons and determine whether a doc matches on `#advance`?
   
   > something like `VectorSimilarityQuery`
   
   I like this, thanks for the suggestion @benwtrent!


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


Re: [PR] Add support for similarity-based vector searches [lucene]

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on PR #12679:
URL: https://github.com/apache/lucene/pull/12679#issuecomment-1769218762

   @kaivalnp I see the issue with my test, you are specifically testing "post-filtering" on the top values, not just getting the top10 k. I understand my issue.
   
   Could you post your testing code or something in a gist ?


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


Re: [PR] Add support for radius-based vector searches [lucene]

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on PR #12679:
URL: https://github.com/apache/lucene/pull/12679#issuecomment-1768734871

   >  Something like a lazy-loading iterator, where we perform vector comparisons and determine whether a doc matches on #advance?
   
   I think @kaivalnp the thing to do would be to say the Collector is full by flagging "incomplete" (I think this is possible) once a threshold is reached. You can do this independently from a "maxvisit" as we don't care about visiting the vector, we just care about adding it to the result set.


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


Re: [PR] Add support for similarity-based vector searches [lucene]

Posted by "kaivalnp (via GitHub)" <gi...@apache.org>.
kaivalnp commented on PR #12679:
URL: https://github.com/apache/lucene/pull/12679#issuecomment-1806196196

   Summary of new changes:
   1. Refactor into a more appropriate query
       - Move away from `AbstractKnnVectorQuery` to take advantage of inherent independence of segment-level results
       - KNN queries need to execute the core logic in `#rewrite` because of an inter-dependence of segment-level results (that is, given N segment-level hits we cannot determine if they will appear in the index-level `topK` without knowing results from other segments). This leads to requirements of [custom concurrency](https://github.com/apache/lucene/blob/main/lucene/core/src/java/org/apache/lucene/search/AbstractKnnVectorQuery.java#L82-L88) for individual HNSW searches, which should ideally be parallel by default
       - We can move graph searches down to a more appropriate place (like `#scorer`) to take advantage of this
   
   2. Return a lazy-loading iterator instead of a greedy exact search (thanks @jpountz!)
       - Introduce a `visitLimit` on the number of nodes to traverse before stopping graph search - deeming it "too expensive". Once this is exhausted, return a lazy-loading iterator on all vectors (functionally equivalent to an exact search)
       - Unlike KNN queries, which need to traverse all vectors to determine which ones are present in the `topK` best-scoring ones, a similarity-based vector search can independently determine if a vector is a result or not (based on whether its similarity with the query is above a `resultSimilarity`)
       - Making use of this behavior, we can prevent a greedy exact search for collecting all matching docs into a list on heap, and determine if a vector is a match inside a `FilteredDocIdSetIterator`
       - This has a huge benefit when the query will be one of the clauses of a `BooleanQuery` (so other clauses will filter out non-matching docs and this query will only compute similarity scores with already filtered vectors). In the worst case, this will consider all vectors (same as exact search)
       - We also have useful information from graph search - mainly which hit was evaluated, and which hit was collected. This information can be re-used from the iterator: if a hit has been traversed, it will either be added to the results, or discarded. If it is present in the results, we simply lookup the score, otherwise mark it as rejected
       - If a vector has not been traversed in graph search, we compute its similarity score (so each query - document pair will only compute similarity scores once)
   
   Please let me know if this approach 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


Re: [PR] Add support for similarity-based vector searches [lucene]

Posted by "kaivalnp (via GitHub)" <gi...@apache.org>.
kaivalnp commented on PR #12679:
URL: https://github.com/apache/lucene/pull/12679#issuecomment-1846106597

   Thanks for all the help here @benwtrent !
   
   > Could you add changes for Lucene 9.10?
   
   Added an entry under "New Features" (also added one of my teammates along with whom this change was designed)


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


Re: [PR] Add support for similarity-based vector searches [lucene]

Posted by "kaivalnp (via GitHub)" <gi...@apache.org>.
kaivalnp closed pull request #12679: Add support for similarity-based vector searches
URL: https://github.com/apache/lucene/pull/12679


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


Re: [PR] Add support for similarity-based vector searches [lucene]

Posted by "kaivalnp (via GitHub)" <gi...@apache.org>.
kaivalnp commented on PR #12679:
URL: https://github.com/apache/lucene/pull/12679#issuecomment-1812956627

   > could you test on cohere with Max-inner product?
   
   Thanks, the gist was really helpful and gave some files including normalized and un-normalized vectors. I assume that since you mentioned `MAXIMUM_INNER_PRODUCT`, you wanted the un-normalized vectors
   
   I saw \~476k vectors of 768 dimensions there and indexed the first 400k in a *single segment*, while querying the next 10k, using the following command:
   
   ```sh
   ./gradlew :lucene:core:similarity-benchmark --args=" --vecPath=/home/kaivalnp/working/similarity-benchmark/cohere-768.vec --indexPath=/home/kaivalnp/working/similarity-benchmark/cohere-indexes --dim=768 --function=MAXIMUM_INNER_PRODUCT --numDocs=400000 --numQueries=10000 --topKs=5000,2500,1000,500,100 --topK-thresholds=300,305,310,315,320 --traversalSimilarities=295,300,305,310,315 --resultSimilarities=300,305,310,315,320"
   ```
   
   ### KNN search
   
   | maxConn | beamWidth | topK | threshold | count   | numVisited | latency | recall |
   | ------- | --------- | ---- | --------- | ------- | ---------- | ------- | ------ |
   | 16      | 100       | 5000 | 300.00    | 1123.19 | 40056.44   | 98.96   | 0.89   |
   | 16      | 100       | 2500 | 305.00    | 480.82  | 23258.29   | 54.91   | 0.83   |
   | 16      | 100       | 1000 | 310.00    | 191.52  | 11249.93   | 26.12   | 0.73   |
   | 16      | 100       | 500  | 315.00    | 83.21   | 6487.60    | 14.87   | 0.69   |
   | 16      | 100       | 100  | 320.00    | 23.80   | 1832.45    | 4.00    | 0.43   |
   | 16      | 200       | 5000 | 300.00    | 1126.33 | 44928.96   | 107.69  | 0.89   |
   | 16      | 200       | 2500 | 305.00    | 482.17  | 26242.83   | 61.47   | 0.83   |
   | 16      | 200       | 1000 | 310.00    | 192.13  | 12751.78   | 29.42   | 0.73   |
   | 16      | 200       | 500  | 315.00    | 83.49   | 7360.26    | 16.67   | 0.70   |
   | 16      | 200       | 100  | 320.00    | 23.89   | 2056.14    | 4.51    | 0.44   |
   | 32      | 100       | 5000 | 300.00    | 1128.81 | 51636.98   | 122.67  | 0.89   |
   | 32      | 100       | 2500 | 305.00    | 483.29  | 30892.01   | 72.01   | 0.84   |
   | 32      | 100       | 1000 | 310.00    | 192.65  | 15424.38   | 35.12   | 0.73   |
   | 32      | 100       | 500  | 315.00    | 83.72   | 9060.78    | 20.28   | 0.70   |
   | 32      | 100       | 100  | 320.00    | 24.00   | 2606.37    | 5.70    | 0.44   |
   | 32      | 200       | 5000 | 300.00    | 1130.18 | 61350.93   | 145.76  | 0.89   |
   | 32      | 200       | 2500 | 305.00    | 483.95  | 37178.70   | 86.05   | 0.84   |
   | 32      | 200       | 1000 | 310.00    | 192.99  | 18778.34   | 42.14   | 0.73   |
   | 32      | 200       | 500  | 315.00    | 83.90   | 11083.97   | 24.54   | 0.70   |
   | 32      | 200       | 100  | 320.00    | 24.08   | 3172.91    | 6.83    | 0.44   |
   | 64      | 100       | 5000 | 300.00    | 1129.81 | 58389.13   | 138.14  | 0.89   |
   | 64      | 100       | 2500 | 305.00    | 483.77  | 35567.55   | 81.62   | 0.84   |
   | 64      | 100       | 1000 | 310.00    | 192.87  | 18093.55   | 40.34   | 0.73   |
   | 64      | 100       | 500  | 315.00    | 83.84   | 10734.50   | 23.76   | 0.70   |
   | 64      | 100       | 100  | 320.00    | 24.06   | 3122.13    | 6.77    | 0.44   |
   | 64      | 200       | 5000 | 300.00    | 1130.78 | 72620.92   | 169.86  | 0.89   |
   | 64      | 200       | 2500 | 305.00    | 484.24  | 45052.36   | 101.93  | 0.84   |
   | 64      | 200       | 1000 | 310.00    | 193.16  | 23283.96   | 51.61   | 0.73   |
   | 64      | 200       | 500  | 315.00    | 83.99   | 13908.95   | 30.44   | 0.70   |
   | 64      | 200       | 100  | 320.00    | 24.13   | 4035.89    | 8.61    | 0.44   |
   
   ### Similarity-based search
   
   | maxConn | beamWidth | traversalSimilarity | resultSimilarity | count   | numVisited | latency | recall |
   | ------- | --------- | ------------------- | ---------------- | ------- | ---------- | ------- | ------ |
   | 16      | 100       | 295.00              | 300.00           | 1209.53 | 18270.70   | 44.38   | 0.95   |
   | 16      | 100       | 300.00              | 305.00           | 538.00  | 8833.17    | 21.02   | 0.93   |
   | 16      | 100       | 305.00              | 310.00           | 239.11  | 4249.13    | 9.97    | 0.91   |
   | 16      | 100       | 310.00              | 315.00           | 105.02  | 2050.95    | 4.87    | 0.87   |
   | 16      | 100       | 315.00              | 320.00           | 45.71   | 1028.26    | 2.35    | 0.83   |
   | 16      | 200       | 295.00              | 300.00           | 1217.74 | 20335.62   | 49.38   | 0.96   |
   | 16      | 200       | 300.00              | 305.00           | 542.19  | 9851.65    | 23.54   | 0.94   |
   | 16      | 200       | 305.00              | 310.00           | 240.68  | 4726.50    | 11.04   | 0.91   |
   | 16      | 200       | 310.00              | 315.00           | 106.02  | 2287.34    | 5.33    | 0.88   |
   | 16      | 200       | 315.00              | 320.00           | 46.09   | 1139.68    | 2.60    | 0.84   |
   | 32      | 100       | 295.00              | 300.00           | 1235.75 | 25159.18   | 59.94   | 0.98   |
   | 32      | 100       | 300.00              | 305.00           | 554.76  | 12709.10   | 29.69   | 0.96   |
   | 32      | 100       | 305.00              | 310.00           | 247.15  | 6275.45    | 14.46   | 0.94   |
   | 32      | 100       | 310.00              | 315.00           | 108.95  | 3093.07    | 7.00    | 0.91   |
   | 32      | 100       | 315.00              | 320.00           | 47.39   | 1544.48    | 3.47    | 0.86   |
   | 32      | 200       | 295.00              | 300.00           | 1243.78 | 29690.87   | 70.66   | 0.98   |
   | 32      | 200       | 300.00              | 305.00           | 558.98  | 15064.99   | 34.99   | 0.97   |
   | 32      | 200       | 305.00              | 310.00           | 249.03  | 7442.06    | 17.09   | 0.95   |
   | 32      | 200       | 310.00              | 315.00           | 110.01  | 3664.88    | 8.28    | 0.92   |
   | 32      | 200       | 315.00              | 320.00           | 47.92   | 1826.35    | 4.06    | 0.87   |
   | 64      | 100       | 295.00              | 300.00           | 1228.98 | 29028.54   | 68.77   | 0.97   |
   | 64      | 100       | 300.00              | 305.00           | 549.09  | 14931.68   | 34.43   | 0.95   |
   | 64      | 100       | 305.00              | 310.00           | 242.41  | 7417.15    | 16.89   | 0.92   |
   | 64      | 100       | 310.00              | 315.00           | 105.26  | 3613.84    | 8.12    | 0.88   |
   | 64      | 100       | 315.00              | 320.00           | 45.14   | 1794.89    | 4.02    | 0.82   |
   | 64      | 200       | 295.00              | 300.00           | 1243.45 | 36266.02   | 85.05   | 0.98   |
   | 64      | 200       | 300.00              | 305.00           | 557.47  | 18811.49   | 42.83   | 0.96   |
   | 64      | 200       | 305.00              | 310.00           | 246.42  | 9377.28    | 21.11   | 0.94   |
   | 64      | 200       | 310.00              | 315.00           | 107.09  | 4559.22    | 10.20   | 0.89   |
   | 64      | 200       | 315.00              | 320.00           | 45.99   | 2249.22    | 4.99    | 0.84   |
   
   **IF** the goal is to "get all vectors above a similarity", then looks like using the new similarity-based search API scales better than having a large `topK` and post-filtering results later


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


Re: [PR] Add support for similarity-based vector searches [lucene]

Posted by "kaivalnp (via GitHub)" <gi...@apache.org>.
kaivalnp commented on PR #12679:
URL: https://github.com/apache/lucene/pull/12679#issuecomment-1834591675

   Thanks @benwtrent! I also simplified the queries:
   
   I realized that the API may be difficult to use in the current state (we are leaving two parameters - `traversalSimilarity` and `visitLimit` upto the user to configure, which may be a large overhead)
   
   I noticed from above benchmarks that `traversalSimilarity` is good for tuning (acts like the `fanout` equivalent of `topK`) but most users need not change this -- and we can keep it equal to `resultSimilarity` by default (but still allow configuring it, whenever required)
   
   Another issue previously encountered (amplified by the above change) is that we stop graph search too early when the entry node is far away from the query. To overcome this, can we continue search as long as we find better scoring nodes (so we know there is a possibility of reaching nodes above `resultSimilarity`)?
   
   For configuring `visitLimit`, seems like the best option is to add a `filter` (like in `AbstractKnnVectorQuery`) - where we determine the `visitLimit` from the cost of the filter, and fall back to exact search over filtered docs - once this limit is reached..
   
   Here is the benchmark setup and results with these changes (same range as before): https://gist.github.com/kaivalnp/07d6a96d22adfad4d3cd5924b13ed524
   
   Also added some tests
   
   > I do worry a bit around the post-filtering. It seems likely in a restrictive search scenario, we would do a bunch of searching to no avail
   
   Agreed, we do some work in graph search (like similarity computations, collecting results, etc) - which should be reusable from exact search
   
   I had opened #12820 to discuss this issue (also affects KNN queries) - perhaps we can include these similarity-based queries if we arrive to a solution there?


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


Re: [PR] Add support for similarity-based vector searches [lucene]

Posted by "kaivalnp (via GitHub)" <gi...@apache.org>.
kaivalnp commented on code in PR #12679:
URL: https://github.com/apache/lucene/pull/12679#discussion_r1411286512


##########
lucene/core/src/java/org/apache/lucene/search/AbstractRnnVectorQuery.java:
##########


Review Comment:
   Added now



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


Re: [PR] Add support for similarity-based vector searches [lucene]

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12679:
URL: https://github.com/apache/lucene/pull/12679#discussion_r1419666959


##########
lucene/CHANGES.txt:
##########
@@ -167,7 +167,10 @@ API Changes
 
 New Features
 ---------------------
-(No changes)
+
+* GITHUB#12679: Add support for similarity-based vector searches. Finds all vectors scoring above a `resultSimilarity`
+  while traversing the HNSW graph till better-scoring nodes are available, or the best candidate is below a score of
+  `traversalSimilarity` in the lowest level. (Aditya Prakash, Kaival Parikh)

Review Comment:
   > Add support for similarity-based vector searches
   
   Well, what are the query names? :D



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


Re: [PR] Add support for radius-based vector searches [lucene]

Posted by "jpountz (via GitHub)" <gi...@apache.org>.
jpountz commented on PR #12679:
URL: https://github.com/apache/lucene/pull/12679#issuecomment-1765975335

   If I read correctly, this query ends up calling `LeafReader#searchNearestNeighbors` with k=Integer.MAX_VALUE, which will not only run in O(maxDoc) time but also use O(maxDoc) memory. I don't think we should do this.
   
   In my opinion, there are two options: either we force this query to take a `k` parameter and make it only return the top `k` nearest neighbors that are also within the radius. Or we make it always run in "exact" mode with a two-phase iterator that performs the similary check in `TwoPhaseIterator#matches()`. We'd then need to prefix this query with `Slow` like other queries that work similarly.


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


Re: [PR] Add support for similarity-based vector searches [lucene]

Posted by "kaivalnp (via GitHub)" <gi...@apache.org>.
kaivalnp commented on PR #12679:
URL: https://github.com/apache/lucene/pull/12679#issuecomment-1769299867

   Here is the gist of my benchmark: https://gist.github.com/kaivalnp/79808017ed7666214540213d1e2a21cf
   
   I'm calculating the baseline / individual results as "count of vectors above the threshold"
   
   Note that we do not need the actual vectors, because any vector with a score >= `resultSimilarity` is implicitly in the baseline. This simplifies the benchmark to just maintaining counts of vectors (as opposed to the actual vector IDs), and recall is calculated as the "ratio of total count of vectors found by KNN or RNN / total count of vectors in the baseline"
   
   Had some other helper functions mainly for calling these and formatting output, but kept the important functions in the gist (how I'm calculating the baseline, KNN / RNN results and time taken)


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


Re: [PR] Add support for similarity-based vector searches [lucene]

Posted by "kaivalnp (via GitHub)" <gi...@apache.org>.
kaivalnp commented on PR #12679:
URL: https://github.com/apache/lucene/pull/12679#issuecomment-1769235109

   Thanks for running this @benwtrent!
   
   I just had a couple of questions:
   1. What was your baseline in the test? If the baseline / goal is to "get the K-Nearest Neighbors", then the threshold-based search is not the best way to achieve it. I believe the true baseline should be *all* vectors *above* the score threshold
   2. If the baseline is KNN, we should at least post-filter results below the threshold. The threshold-based search will never be able to find results below the `resultSimilarity`, and it may not be a fair comparison
   
   As I'm writing this, I see your comment. I'll post my setup in a while


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


Re: [PR] Add support for similarity-based vector searches [lucene]

Posted by "kaivalnp (via GitHub)" <gi...@apache.org>.
kaivalnp commented on PR #12679:
URL: https://github.com/apache/lucene/pull/12679#issuecomment-1779796454

   Hi @benwtrent! Curious to hear if you've been able to reproduce the benchmark?


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


Re: [PR] Add support for radius-based vector searches [lucene]

Posted by "kaivalnp (via GitHub)" <gi...@apache.org>.
kaivalnp commented on PR #12679:
URL: https://github.com/apache/lucene/pull/12679#issuecomment-1766995337

   ### Benchmarks
   
   Using the vector file from https://home.apache.org/~sokolov/enwiki-20120502-lines-1k-100d.vec (enwiki dataset, unit vectors, 100 dimensions)
   
   The setup was 1M doc vectors in a single HNSW graph with `DOT_PRODUCT` similarity, and 10K query vectors
   
   The baseline for the new objective is "all vectors above a score threshold" (as opposed to the best-scoring `topK` vectors in the current system) for a given query and is used to compute recall in all subsequent runs..
   
   Here are some statistics for the result counts in the new baseline:
   
   | threshold | mean     | stdDev    | min | p25 | p50  | p75    | p90    | p99    | max    |
   | --------- | -------- | --------- | --- | --- | ---- | ------ | ------ | ------ | ------ |
   | 0.95      | 71877.73 | 109177.23 | 0   | 222 | 7436 | 116567 | 259135 | 388113 | 483330 |
   | 0.96      | 32155.63 | 57183.83  | 0   | 30  | 3524 | 36143  | 120700 | 235038 | 342959 |
   | 0.97      | 8865.48  | 19006.24  | 0   | 1   | 816  | 5483   | 29966  | 92433  | 174163 |
   | 0.98      | 1010.10  | 2423.03   | 0   | 0   | 46   | 873    | 3234   | 12175  | 40163  |
   | 0.99      | 136.47   | 465.91    | 0   | 0   | 0    | 2      | 77     | 2296   | 2494   |
   
   This is used to get an estimate of query - result count distribution for various `threshold` values, and also gauge the corresponding `topK` to use for comparison with the new radius-based vector search API
   
   Here we will benchmark the new API against a high `topK` (+ filtering out results below the threshold after HNSW search)
   
   ### K-NN Search (current system)
   
   | maxConn | beamWidth | topK | threshold | mean   | numVisited | latency | recall |
   | ------- | --------- | ---- | --------- | ------ | ---------- | ------- | ------ |
   | 16      | 100       | 500  | 0.99      | 46.39  | 4086       | 1.465   | 0.34   |
   | 16      | 100       | 1000 | 0.99      | 83.92  | 6890       | 2.600   | 0.61   |
   | 16      | 100       | 2000 | 0.99      | 129.56 | 11727      | 4.746   | 0.95   |
   | 16      | 200       | 500  | 0.99      | 46.39  | 4504       | 1.535   | 0.34   |
   | 16      | 200       | 1000 | 0.99      | 83.92  | 7564       | 2.759   | 0.61   |
   | 16      | 200       | 2000 | 0.99      | 129.56 | 12805      | 5.007   | 0.95   |
   | 32      | 100       | 500  | 0.99      | 46.39  | 4940       | 1.644   | 0.34   |
   | 32      | 100       | 1000 | 0.99      | 83.92  | 8271       | 2.944   | 0.61   |
   | 32      | 100       | 2000 | 0.99      | 129.56 | 13937      | 5.335   | 0.95   |
   | 32      | 200       | 500  | 0.99      | 46.39  | 5654       | 1.890   | 0.34   |
   | 32      | 200       | 1000 | 0.99      | 83.92  | 9401       | 3.320   | 0.61   |
   | 32      | 200       | 2000 | 0.99      | 129.56 | 15707      | 5.987   | 0.95   |
   | 64      | 100       | 500  | 0.99      | 46.39  | 5241       | 1.736   | 0.34   |
   | 64      | 100       | 1000 | 0.99      | 83.92  | 8766       | 3.091   | 0.61   |
   | 64      | 100       | 2000 | 0.99      | 129.56 | 14736      | 5.567   | 0.95   |
   | 64      | 200       | 500  | 0.99      | 46.39  | 6095       | 1.992   | 0.34   |
   | 64      | 200       | 1000 | 0.99      | 83.92  | 10119      | 3.535   | 0.61   |
   | 64      | 200       | 2000 | 0.99      | 129.56 | 16852      | 6.365   | 0.95   |
   
   ### R-NN Search (new system)
   
   | maxConn | beamWidth | traversalThreshold | threshold | mean   | numVisited | latency | recall |
   | ------- | --------- | ------------------ | --------- | ------ | ---------- | ------- | ------ |
   | 16      | 100       | 0.99               | 0.99      | 94.03  | 256        | 0.129   | 0.69   |
   | 16      | 100       | 0.98               | 0.99      | 95.18  | 5171       | 2.062   | 0.70   |
   | 16      | 200       | 0.99               | 0.99      | 89.96  | 263        | 0.119   | 0.66   |
   | 16      | 200       | 0.98               | 0.99      | 91.09  | 5497       | 2.207   | 0.67   |
   | 32      | 100       | 0.99               | 0.99      | 109.17 | 295        | 0.135   | 0.80   |
   | 32      | 100       | 0.98               | 0.99      | 110.89 | 6529       | 2.580   | 0.81   |
   | 32      | 200       | 0.99               | 0.99      | 108.97 | 313        | 0.142   | 0.80   |
   | 32      | 200       | 0.98               | 0.99      | 110.55 | 7145       | 2.861   | 0.81   |
   | 64      | 100       | 0.99               | 0.99      | 133.61 | 314        | 0.152   | 0.98   |
   | 64      | 100       | 0.98               | 0.99      | 135.74 | 7033       | 2.765   | 0.99   |
   | 64      | 200       | 0.99               | 0.99      | 133.84 | 333        | 0.163   | 0.98   |
   | 64      | 200       | 0.98               | 0.99      | 135.96 | 7833       | 3.121   | 1.00   |
   
   - `mean` is the average number of results above the `threshold`
   - `numVisited` is the average number of HNSW nodes visited per-query
   - The latency is measured in `ms` per-query
   
   **IF** the goal is to "get all vectors within a radius", then looks like using the new radius-based search API scales better than having a large `topK` and post-filtering results later?
   


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


Re: [PR] Add support for similarity-based vector searches [lucene]

Posted by "kaivalnp (via GitHub)" <gi...@apache.org>.
kaivalnp commented on PR #12679:
URL: https://github.com/apache/lucene/pull/12679#issuecomment-1809743395

   ### Benchmark Setup
   
   Sharing my benchmark setup for reproducibility in [this branch](https://github.com/kaivalnp/lucene/tree/similarity-benchmark) (see [this commit](https://github.com/apache/lucene/commit/41610c734308505c87018f2da3343a489a7f7a59) on top of changes in the PR)
   
   *Consideration*: Being a similarity-based search, we need not maintain the actual results above a `resultSimilarity` (because any hit above this threshold is implicitly correct, and in the "golden set / baseline"). This makes recall computations easier: simply maintain per-query average count of results above the `resultSimilarity`, and a ratio of candidate / baseline gives the "recall"
   
   Added a utility class with some command line arguments:
   - `--vecPath`: path to VEC file with vectors arranged one after another, in `LITTLE_ENDIAN` byte order (example https://home.apache.org/~sokolov/enwiki-20120502-lines-1k-100d.vec)
   - `--dim`: dimension of vectors, default `100` from above file
   - `--numDocs`: number of vectors to index, default `1M`
   - `--numQueries`: number of vectors to search for, default `10K`
   - `--indexPath`: a parent directory for storing multiple indexes (of `maxConn` - `beamWidth` combinations. Also takes `numDocs`, `knnField`, `function` and `maxNumSegments` into account for naming)
   - `--knnField`: name of KNN field, default `knn`
   - `--function`: `VectorSimilarityFunction` to use (one of `[EUCLIDEAN, DOT_PRODUCT, COSINE, MAXIMUM_INNER_PRODUCT]`), default `DOT_PRODUCT`
   - `--maxNumSegments`: number of segments to force merge into, default `1` to mitigate noise of parallelism
   - `--maxConns`: list of `maxConn` values to try, default `{16, 32, 64}`
   - `--beamWidths`: list of `beamWidth` values to try, default `{100, 200}`
   - `--topKs`: list of `topK` values to try in the KNN-based search, default to none
   - `--topK-thresholds`: list of `threshold` values to post-filter in the KNN-based search, default to none. 1-1 correspondence with `topKs` parameter (must be of same size)
   - `--traversalSimilarities`: list of `traversalSimilarity` values to try in the similarity-based search, default to none
   - `--resultSimilarities`: list of `resultSimilarity` values to try in the similarity-based search, default to none. 1-1 correspondence with `traversalSimilarities` parameter (must be of same size)
   
   The script takes the first `numDocs` vectors and creates multiple indexes in the `indexPath` parent directory with `dim-numDocs-knnField-function-maxNumSegments-maxConn-beamWidth` folder name. It takes the next `numQueries` vectors as queries
   
   It goes over all `topKs` - `topK-thresholds` and performs KNN-based search, also computing and caching the "true" results for recall (to be reused from similarity-based search). Uses a `KnnFloatVectorQuery`, but removes results below the `topK-threshold` after graph search. The idea is to simulate a high `topK` + post-filtering
   
   It then goes over all `traversalSimilarities` - `resultSimilarities` and performs similarity-based search, re-using or computing the "true" results for recall
   
   Finally prints both results in a formatted table (shown below)
   
   The script can be executed using `./gradlew :lucene:core:similarity-benchmark`. It makes use of parallelism wherever possible to speed things up (each query on a different thread) - *please set the desired concurrency based on your machine*. The latency reporting takes multi-threading into account and averages times for each search in a single thread (even when multiple are running in parallel and wall-clock time may be lower)
   
   Here is a sample command on my machine, and corresponding output (took \~30 min *after all indexes were created*):
   
   ```sh
   ./gradlew :lucene:core:similarity-benchmark -Djava.util.concurrent.ForkJoinPool.common.parallelism=64 --args="--vecPath=/home/kaivalnp/working/similarity-benchmark/docs.vec --indexPath=/home/kaivalnp/working/similarity-benchmark/indexes --topKs=500,1000,2000 --topK-thresholds=0.99,0.99,0.99 --traversalSimilarities=0.98,0.99 --resultSimilarities=0.99,0.99"
   ```
   
   ### KNN search
   
   | maxConn | beamWidth | topK | threshold | count  | numVisited | latency | recall |
   | ------- | --------- | ---- | --------- | ------ | ---------- | ------- | ------ |
   | 16      | 100       | 500  | 0.99      | 46.39  | 4086.90    | 2.58    | 0.34   |
   | 16      | 100       | 1000 | 0.99      | 83.92  | 6890.43    | 4.50    | 0.61   |
   | 16      | 100       | 2000 | 0.99      | 129.56 | 11727.72   | 8.09    | 0.95   |
   | 16      | 200       | 500  | 0.99      | 46.39  | 4504.86    | 2.72    | 0.34   |
   | 16      | 200       | 1000 | 0.99      | 83.92  | 7564.04    | 4.89    | 0.61   |
   | 16      | 200       | 2000 | 0.99      | 129.56 | 12805.53   | 8.86    | 0.95   |
   | 32      | 100       | 500  | 0.99      | 46.39  | 4940.79    | 2.94    | 0.34   |
   | 32      | 100       | 1000 | 0.99      | 83.92  | 8271.67    | 5.26    | 0.61   |
   | 32      | 100       | 2000 | 0.99      | 129.56 | 13937.63   | 9.40    | 0.95   |
   | 32      | 200       | 500  | 0.99      | 46.39  | 5654.39    | 3.35    | 0.34   |
   | 32      | 200       | 1000 | 0.99      | 83.92  | 9401.15    | 6.02    | 0.61   |
   | 32      | 200       | 2000 | 0.99      | 129.56 | 15707.08   | 10.71   | 0.95   |
   | 64      | 100       | 500  | 0.99      | 46.39  | 5241.29    | 3.09    | 0.34   |
   | 64      | 100       | 1000 | 0.99      | 83.92  | 8766.76    | 5.53    | 0.61   |
   | 64      | 100       | 2000 | 0.99      | 129.56 | 14736.85   | 9.86    | 0.95   |
   | 64      | 200       | 500  | 0.99      | 46.40  | 6095.06    | 3.57    | 0.34   |
   | 64      | 200       | 1000 | 0.99      | 83.92  | 10119.29   | 6.36    | 0.61   |
   | 64      | 200       | 2000 | 0.99      | 129.56 | 16852.31   | 11.27   | 0.95   |
   
   ### Similarity-based search
   
   | maxConn | beamWidth | traversalSimilarity | resultSimilarity | count  | numVisited | latency | recall |
   | ------- | --------- | ------------------- | ---------------- | ------ | ---------- | ------- | ------ |
   | 16      | 100       | 0.98                | 0.99             | 95.18  | 5171.94    | 3.62    | 0.70   |
   | 16      | 100       | 0.99                | 0.99             | 94.03  | 256.82     | 0.22    | 0.69   |
   | 16      | 200       | 0.98                | 0.99             | 91.09  | 5497.00    | 3.88    | 0.67   |
   | 16      | 200       | 0.99                | 0.99             | 89.96  | 263.99     | 0.23    | 0.66   |
   | 32      | 100       | 0.98                | 0.99             | 110.89 | 6529.97    | 4.52    | 0.81   |
   | 32      | 100       | 0.99                | 0.99             | 109.17 | 295.06     | 0.26    | 0.80   |
   | 32      | 200       | 0.98                | 0.99             | 110.55 | 7145.58    | 5.04    | 0.81   |
   | 32      | 200       | 0.99                | 0.99             | 108.97 | 313.19     | 0.27    | 0.80   |
   | 64      | 100       | 0.98                | 0.99             | 135.74 | 7033.33    | 4.87    | 0.99   |
   | 64      | 100       | 0.99                | 0.99             | 133.61 | 314.55     | 0.30    | 0.98   |
   | 64      | 200       | 0.98                | 0.99             | 135.96 | 7833.15    | 5.52    | 1.00   |
   | 64      | 200       | 0.99                | 0.99             | 133.84 | 333.15     | 0.31    | 0.98   |
   
   The `count`, `numVisited`, `recall` are exactly same as before. There is a difference in baseline of `latency` because of running a different machine, but the ratio is about the same..
   
   Please let me know if you're able to reproduce it / run it on a different vector file @benwtrent..
   


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


Re: [PR] Add support for similarity-based vector searches [lucene]

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on PR #12679:
URL: https://github.com/apache/lucene/pull/12679#issuecomment-1779866529

   @kaivalnp I have been busy doing other things. I hope to look into this in the next week or so.


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


Re: [PR] Add support for similarity-based vector searches [lucene]

Posted by "kaivalnp (via GitHub)" <gi...@apache.org>.
kaivalnp commented on code in PR #12679:
URL: https://github.com/apache/lucene/pull/12679#discussion_r1389857835


##########
lucene/core/src/java/org/apache/lucene/search/VectorSimilarityCollector.java:
##########
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import org.apache.lucene.util.BitSet;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.SparseFixedBitSet;
+
+/**
+ * Perform a similarity-based graph search.
+ *
+ * @lucene.experimental
+ */
+class VectorSimilarityCollector extends AbstractKnnCollector {
+  private static final Bits MATCH_ALL_BITS = new Bits.MatchAllBits(Integer.MAX_VALUE);
+
+  private final float traversalSimilarity, resultSimilarity;
+  private final List<ScoreDoc> scoreDocList;
+  private final BitSet visited;
+
+  /**
+   * Perform a similarity-based graph search. All nodes above a {@link #traversalSimilarity} are
+   * traversed, and all nodes above a {@link #resultSimilarity} are collected.
+   *
+   * @param traversalSimilarity (lower) similarity score for graph traversal.
+   * @param resultSimilarity (higher) similarity score for result collection.
+   * @param visitLimit limit on number of nodes to visit.
+   * @param maxDoc maximum docid of any node.
+   */
+  public VectorSimilarityCollector(
+      float traversalSimilarity, float resultSimilarity, long visitLimit, int maxDoc) {
+    super(1, visitLimit);
+    if (traversalSimilarity > resultSimilarity) {
+      throw new IllegalArgumentException("traversalSimilarity should be <= resultSimilarity");
+    }
+    this.traversalSimilarity = traversalSimilarity;
+    this.resultSimilarity = resultSimilarity;
+    this.scoreDocList = new ArrayList<>();
+
+    if (visitLimit == Long.MAX_VALUE) {
+      this.visited = null;
+    } else {
+      this.visited = new SparseFixedBitSet(maxDoc);
+    }

Review Comment:
   The `SparseFixedBitSet` has a larger purpose than counting nodes: During graph traversal, we have visited some nodes based on `traversalSimilarity` but accepted them on the basis of `resultSimilarity` [here](https://github.com/apache/lucene/blob/cad565439be512ac6e95a698007b1fc971173f00/lucene/core/src/java/org/apache/lucene/search/VectorSimilarityCollector.java#L70-L73)
   
   That is, we know that all nodes "visited but not collected" are below the `resultSimilarity`. I'm using this information [here](https://github.com/apache/lucene/blob/cad565439be512ac6e95a698007b1fc971173f00/lucene/core/src/java/org/apache/lucene/search/AbstractVectorSimilarityQuery.java#L115-L121) to reject such docs
   
   This is done to prevent duplicate computation of similarity scores for docs we already know are ineligible



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


Re: [PR] Add support for similarity-based vector searches [lucene]

Posted by "kaivalnp (via GitHub)" <gi...@apache.org>.
kaivalnp commented on PR #12679:
URL: https://github.com/apache/lucene/pull/12679#issuecomment-1811057058

   Keeping the `visitLimit` = 0 (immediately fallback to lazy iterator) we expect an exact search to be performed (and `recall` = 1) as soon as the first node is visited (`numVisited` = 1), although at the cost of a full vector search (latency is two orders of magnitude higher - number of similarity computations will be \~1M)
   
   | maxConn | beamWidth | traversalSimilarity | resultSimilarity | count  | numVisited | latency | recall |
   | ------- | --------- | ------------------- | ---------------- | ------ | ---------- | ------- | ------ |
   | 16      | 100       | 0.99                | 0.99             | 136.47 | 1.00       | 268.23  | 1.00   |
   | 16      | 200       | 0.99                | 0.99             | 136.47 | 1.00       | 269.01  | 1.00   |
   | 32      | 100       | 0.99                | 0.99             | 136.47 | 1.00       | 275.00  | 1.00   |
   | 32      | 200       | 0.99                | 0.99             | 136.47 | 1.00       | 268.45  | 1.00   |
   | 64      | 100       | 0.99                | 0.99             | 136.47 | 1.00       | 270.93  | 1.00   |
   | 64      | 200       | 0.99                | 0.99             | 136.47 | 1.00       | 270.63  | 1.00   |
   
   This nicely emphasizes the need to pick a sensible `visitLimit` to not fallback prematurely since it can become very expensive, and also verifies that the lazy iterator works as intended..
   


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


Re: [PR] Add support for radius-based vector searches [lucene]

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on PR #12679:
URL: https://github.com/apache/lucene/pull/12679#issuecomment-1768737424

   The results: https://github.com/apache/lucene/pull/12679#issuecomment-1766995337
   
   Are astounding! I will try and replicate with Lucene Util.
   
   The numbers seem almost too good ;)
   
   


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


Re: [PR] Add support for similarity-based vector searches [lucene]

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on PR #12679:
URL: https://github.com/apache/lucene/pull/12679#issuecomment-1769123216

   OK, I tried testing with KnnGraphTester. 
   
   I indexed 100_000 normalized Cohere vectors (768 dims).
   
   With regular knn, recall@10:
   
   ```
   recall	latency	nDoc	fanout	maxConn	beamWidth	visited	
   0.771	 0.13	100000	0	16	100	        10	
   0.870	 0.19	100000	10	16	100	        20	
   0.953	 0.42	100000	50	16	100	        60	
   0.971	 0.67	100000	100	16	100	        110	
   ```
   
   I tried the similarity threshold and its way worse.
   ```
   recall	latency	nDoc    resultSim travSim  maxConn beamWidth	visited	
   0.889	18.87	100000	0.89000	0.89500	   16	   100	        6714
   0.889	19.69	100000	0.89000	0.89000	   16	   100	        10332
   ```


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


Re: [PR] Add support for similarity-based vector searches [lucene]

Posted by "junqiu-lei (via GitHub)" <gi...@apache.org>.
junqiu-lei commented on PR #12679:
URL: https://github.com/apache/lucene/pull/12679#issuecomment-1907152593

   Hi, do we have any scheduled release date for this exciting feature?


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


Re: [PR] Add support for radius-based vector searches [lucene]

Posted by "kaivalnp (via GitHub)" <gi...@apache.org>.
kaivalnp commented on code in PR #12679:
URL: https://github.com/apache/lucene/pull/12679#discussion_r1362474206


##########
lucene/core/src/java/org/apache/lucene/search/AbstractRnnVectorQuery.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.LeafReaderContext;
+
+/**
+ * Search for all (approximate) vectors within a radius using the {@link RnnCollector}.
+ *
+ * @lucene.experimental
+ */
+abstract class AbstractRnnVectorQuery extends AbstractKnnVectorQuery {
+  private static final TopDocs NO_RESULTS = TopDocsCollector.EMPTY_TOPDOCS;
+
+  protected final float traversalThreshold, resultThreshold;
+
+  public AbstractRnnVectorQuery(
+      String field, float traversalThreshold, float resultThreshold, Query filter) {
+    super(field, Integer.MAX_VALUE, filter);
+    assert traversalThreshold <= resultThreshold;

Review Comment:
   Agreed! `traversalThreshold > resultThreshold` does not make sense, and we want to warn users about this more strongly. Changed now



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


Re: [PR] Add support for similarity-based vector searches [lucene]

Posted by "kaivalnp (via GitHub)" <gi...@apache.org>.
kaivalnp commented on PR #12679:
URL: https://github.com/apache/lucene/pull/12679#issuecomment-1768958493

   Sorry for the confusion, I tried renaming the branch from `radius-based-vector-search` to `similarity-based-vector-search` and the PR closed automatically. I guess I'm stuck with this branch name :(


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


Re: [PR] Add support for similarity-based vector searches [lucene]

Posted by "epotyom (via GitHub)" <gi...@apache.org>.
epotyom commented on PR #12679:
URL: https://github.com/apache/lucene/pull/12679#issuecomment-1851062374

   I see random test failures that could be related to this change:
   
   ```
      >     java.lang.ArrayIndexOutOfBoundsException: Index -1 out of bounds for length 123
      >         at __randomizedtesting.SeedInfo.seed([119135B1F0803918:13366D71F5841AB]:0)
      >         at org.apache.lucene.codecs.simpletext.SimpleTextKnnVectorsReader$SimpleTextFloatVectorValues.vectorValue(SimpleTextKnnVectorsReader.java:346)
      >         at org.apache.lucene.search.VectorScorer$FloatVectorScorer.score(VectorScorer.java:120)
      >         at org.apache.lucene.search.AbstractVectorSimilarityQuery$VectorSimilarityScorer$2.match(AbstractVectorSimilarityQuery.java:259)
      >         at org.apache.lucene.search.FilteredDocIdSetIterator.nextDoc(FilteredDocIdSetIterator.java:64)
      >         at org.apache.lucene.search.Weight$DefaultBulkScorer.scoreRange(Weight.java:269)
      >         at org.apache.lucene.search.Weight$DefaultBulkScorer.score(Weight.java:238)
      >         at org.apache.lucene.tests.search.AssertingBulkScorer.score(AssertingBulkScorer.java:101)
      >         at org.apache.lucene.search.TimeLimitingBulkScorer.score(TimeLimitingBulkScorer.java:82)
      >         at org.apache.lucene.search.BulkScorer.score(BulkScorer.java:38)
      >         at org.apache.lucene.search.IndexSearcher.search(IndexSearcher.java:684)
      >         at org.apache.lucene.tests.search.AssertingIndexSearcher.search(AssertingIndexSearcher.java:79)
      >         at org.apache.lucene.search.IndexSearcher.lambda$search$2(IndexSearcher.java:636)
      >         at org.apache.lucene.search.TaskExecutor$TaskGroup.lambda$createTask$0(TaskExecutor.java:118)
      >         at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
      >         at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
      >         at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
      >         at java.base/java.lang.Thread.run(Thread.java:840)
     2> NOTE: reproduce with: gradlew test --tests TestFloatVectorSimilarityQuery.testRandomFilter -Dtests.seed=119135B1F0803918 -Dtests.locale=mer-Latn-KE -Dtests.timezone=Australia/South -Dtests.asserts=true -Dtests.file.encoding=UTF-8
      >     java.lang.UnsupportedOperationException
      >         at __randomizedtesting.SeedInfo.seed([119135B1F0803918:19473B03FDAEADE7]:0)
      >         at org.apache.lucene.search.TestFloatVectorSimilarityQuery$1.createVectorScorer(TestFloatVectorSimilarityQuery.java:82)
      >         at org.apache.lucene.search.AbstractVectorSimilarityQuery$1.scorer(AbstractVectorSimilarityQuery.java:148)
      >         at org.apache.lucene.search.Weight.scorerSupplier(Weight.java:135)
      >         at org.apache.lucene.search.Weight.bulkScorer(Weight.java:167)
      >         at org.apache.lucene.search.LRUQueryCache$CachingWrapperWeight.cache(LRUQueryCache.java:708)
      >         at org.apache.lucene.search.LRUQueryCache$CachingWrapperWeight.bulkScorer(LRUQueryCache.java:927)
      >         at org.apache.lucene.tests.search.AssertingWeight.bulkScorer(AssertingWeight.java:122)
      >         at org.apache.lucene.search.LRUQueryCache$CachingWrapperWeight.cache(LRUQueryCache.java:708)
      >         at org.apache.lucene.search.LRUQueryCache$CachingWrapperWeight.bulkScorer(LRUQueryCache.java:927)
      >         at org.apache.lucene.tests.search.AssertingWeight.bulkScorer(AssertingWeight.java:122)
      >         at org.apache.lucene.search.IndexSearcher.search(IndexSearcher.java:678)
      >         at org.apache.lucene.tests.search.AssertingIndexSearcher.search(AssertingIndexSearcher.java:79)
      >         at org.apache.lucene.search.IndexSearcher.lambda$search$2(IndexSearcher.java:636)
      >         at org.apache.lucene.search.TaskExecutor$TaskGroup.lambda$createTask$0(TaskExecutor.java:118)
      >         at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
      >         at org.apache.lucene.search.TaskExecutor$TaskGroup.invokeAll(TaskExecutor.java:153)
      >         at org.apache.lucene.search.TaskExecutor.invokeAll(TaskExecutor.java:76)
      >         at org.apache.lucene.search.IndexSearcher.search(IndexSearcher.java:640)
      >         at org.apache.lucene.search.IndexSearcher.search(IndexSearcher.java:607)
      >         at org.apache.lucene.search.IndexSearcher.count(IndexSearcher.java:423)
      >         at org.apache.lucene.search.BaseVectorSimilarityQueryTestCase.testApproximate(BaseVectorSimilarityQueryTestCase.java:460)
      >         at org.apache.lucene.search.TestFloatVectorSimilarityQuery.testApproximate(TestFloatVectorSimilarityQuery.java:26)
      >         at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
      >         at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)
      >         at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
      >         at java.base/java.lang.reflect.Method.invoke(Method.java:568)
      >         at com.carrotsearch.randomizedtesting.RandomizedRunner.invoke(RandomizedRunner.java:1758)
      >         at com.carrotsearch.randomizedtesting.RandomizedRunner$8.evaluate(RandomizedRunner.java:946)
      >         at com.carrotsearch.randomizedtesting.RandomizedRunner$9.evaluate(RandomizedRunner.java:982)
      >         at com.carrotsearch.randomizedtesting.RandomizedRunner$10.evaluate(RandomizedRunner.java:996)
      >         at org.apache.lucene.tests.util.TestRuleSetupTeardownChained$1.evaluate(TestRuleSetupTeardownChained.java:48)
      >         at org.apache.lucene.tests.util.AbstractBeforeAfterRule$1.evaluate(AbstractBeforeAfterRule.java:43)
      >         at org.apache.lucene.tests.util.TestRuleThreadAndTestName$1.evaluate(TestRuleThreadAndTestName.java:45)
      >         at org.apache.lucene.tests.util.TestRuleIgnoreAfterMaxFailures$1.evaluate(TestRuleIgnoreAfterMaxFailures.java:60)
      >         at org.apache.lucene.tests.util.TestRuleMarkFailure$1.evaluate(TestRuleMarkFailure.java:44)
      >         at org.junit.rules.RunRules.evaluate(RunRules.java:20)
      >         at com.carrotsearch.randomizedtesting.rules.StatementAdapter.evaluate(StatementAdapter.java:36)
      >         at com.carrotsearch.randomizedtesting.ThreadLeakControl$StatementRunner.run(ThreadLeakControl.java:390)
      >         at com.carrotsearch.randomizedtesting.ThreadLeakControl.forkTimeoutingTask(ThreadLeakControl.java:843)
      >         at com.carrotsearch.randomizedtesting.ThreadLeakControl$3.evaluate(ThreadLeakControl.java:490)
      >         at com.carrotsearch.randomizedtesting.RandomizedRunner.runSingleTest(RandomizedRunner.java:955)
      >         at com.carrotsearch.randomizedtesting.RandomizedRunner$5.evaluate(RandomizedRunner.java:840)
      >         at com.carrotsearch.randomizedtesting.RandomizedRunner$6.evaluate(RandomizedRunner.java:891)
      >         at com.carrotsearch.randomizedtesting.RandomizedRunner$7.evaluate(RandomizedRunner.java:902)
      >         at org.apache.lucene.tests.util.AbstractBeforeAfterRule$1.evaluate(AbstractBeforeAfterRule.java:43)
      >         at com.carrotsearch.randomizedtesting.rules.StatementAdapter.evaluate(StatementAdapter.java:36)
      >         at org.apache.lucene.tests.util.TestRuleStoreClassName$1.evaluate(TestRuleStoreClassName.java:38)
      >         at com.carrotsearch.randomizedtesting.rules.NoShadowingOrOverridesOnMethodsRule$1.evaluate(NoShadowingOrOverridesOnMethodsRule.java:40)
      >         at com.carrotsearch.randomizedtesting.rules.NoShadowingOrOverridesOnMethodsRule$1.evaluate(NoShadowingOrOverridesOnMethodsRule.java:40)
      >         at com.carrotsearch.randomizedtesting.rules.StatementAdapter.evaluate(StatementAdapter.java:36)
      >         at com.carrotsearch.randomizedtesting.rules.StatementAdapter.evaluate(StatementAdapter.java:36)
      >         at org.apache.lucene.tests.util.TestRuleAssertionsRequired$1.evaluate(TestRuleAssertionsRequired.java:53)
      >         at org.apache.lucene.tests.util.AbstractBeforeAfterRule$1.evaluate(AbstractBeforeAfterRule.java:43)
      >         at org.apache.lucene.tests.util.TestRuleMarkFailure$1.evaluate(TestRuleMarkFailure.java:44)
      >         at org.apache.lucene.tests.util.TestRuleIgnoreAfterMaxFailures$1.evaluate(TestRuleIgnoreAfterMaxFailures.java:60)
      >         at org.apache.lucene.tests.util.TestRuleIgnoreTestSuites$1.evaluate(TestRuleIgnoreTestSuites.java:47)
      >         at org.junit.rules.RunRules.evaluate(RunRules.java:20)
      >         at com.carrotsearch.randomizedtesting.rules.StatementAdapter.evaluate(StatementAdapter.java:36)
      >         at com.carrotsearch.randomizedtesting.ThreadLeakControl$StatementRunner.run(ThreadLeakControl.java:390)
      >         at com.carrotsearch.randomizedtesting.ThreadLeakControl.lambda$forkTimeoutingTask$0(ThreadLeakControl.java:850)
      >         at java.base/java.lang.Thread.run(Thread.java:840)
     2> NOTE: reproduce with: gradlew test --tests TestFloatVectorSimilarityQuery.testApproximate -Dtests.seed=119135B1F0803918 -Dtests.locale=mer-Latn-KE -Dtests.timezone=Australia/South -Dtests.asserts=true -Dtests.file.encoding=UTF-8
   ```


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


Re: [PR] Add support for similarity-based vector searches [lucene]

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12679:
URL: https://github.com/apache/lucene/pull/12679#discussion_r1393228359


##########
lucene/core/src/java/org/apache/lucene/search/AbstractVectorSimilarityQuery.java:
##########
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.Objects;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.util.Bits;
+
+/**
+ * Search for all (approximate) vectors above a similarity threshold.
+ *
+ * @lucene.experimental
+ */
+abstract class AbstractVectorSimilarityQuery extends Query {
+  protected final String field;
+  protected final float traversalSimilarity, resultSimilarity;
+  protected final long visitLimit;
+
+  /**
+   * Search for all (approximate) vectors above a similarity threshold. First performs a
+   * similarity-based graph search using {@link VectorSimilarityCollector} between {@link
+   * #traversalSimilarity} and {@link #resultSimilarity}. If this does not complete within a
+   * specified {@link #visitLimit}, returns a lazy-loading iterator over all vectors above the
+   * {@link #resultSimilarity}.
+   *
+   * @param field a field that has been indexed as a vector field.
+   * @param traversalSimilarity (lower) similarity score for graph traversal.
+   * @param resultSimilarity (higher) similarity score for result collection.
+   * @param visitLimit limit on number of nodes to visit before falling back to a lazy-loading
+   *     iterator.
+   */
+  AbstractVectorSimilarityQuery(
+      String field, float traversalSimilarity, float resultSimilarity, long visitLimit) {
+    this.field = Objects.requireNonNull(field, "field");
+    this.traversalSimilarity = traversalSimilarity;
+    this.resultSimilarity = resultSimilarity;
+    this.visitLimit = visitLimit;
+  }
+
+  abstract VectorScorer createVectorScorer(LeafReaderContext context) throws IOException;
+
+  protected abstract void approximateSearch(LeafReaderContext context, KnnCollector collector)
+      throws IOException;

Review Comment:
   If the user provides a filter where `iterator.cost() < numVectors()`, we do all this work in approximateSearch (because we need to score the values) only to throw it away. I guess the way around this is for users to provide a sane "visitLimit" to prevent too much work from happening.
   
   I am not sure another way around this other than providing a pre-filter. Perhaps the cost is acceptable.



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


Re: [PR] Add support for similarity-based vector searches [lucene]

Posted by "shubhamvishu (via GitHub)" <gi...@apache.org>.
shubhamvishu commented on code in PR #12679:
URL: https://github.com/apache/lucene/pull/12679#discussion_r1367106152


##########
lucene/core/src/java/org/apache/lucene/search/AbstractRnnVectorQuery.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.LeafReaderContext;
+
+/**
+ * Search for all (approximate) vectors within a radius using the {@link RnnCollector}.
+ *
+ * @lucene.experimental
+ */
+abstract class AbstractRnnVectorQuery extends AbstractKnnVectorQuery {
+  private static final TopDocs NO_RESULTS = TopDocsCollector.EMPTY_TOPDOCS;
+
+  protected final float traversalThreshold, resultThreshold;
+
+  public AbstractRnnVectorQuery(
+      String field, float traversalThreshold, float resultThreshold, Query filter) {
+    super(field, Integer.MAX_VALUE, filter);
+    assert traversalThreshold <= resultThreshold;

Review Comment:
   Thanks!



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


Re: [PR] Add support for similarity-based vector searches [lucene]

Posted by "kaivalnp (via GitHub)" <gi...@apache.org>.
kaivalnp commented on PR #12679:
URL: https://github.com/apache/lucene/pull/12679#issuecomment-1907958716

   This feature will ship with Lucene 9.10
   
   I'm not sure when that will be released, though [I see](https://lucene.apache.org/core/corenews.html) \~2-4 months between previous minor versions


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


Re: [PR] Add support for radius-based vector searches [lucene]

Posted by "kaivalnp (via GitHub)" <gi...@apache.org>.
kaivalnp commented on code in PR #12679:
URL: https://github.com/apache/lucene/pull/12679#discussion_r1362472568


##########
lucene/core/src/java/org/apache/lucene/search/AbstractRnnVectorQuery.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.LeafReaderContext;
+
+/**
+ * Search for all (approximate) vectors within a radius using the {@link RnnCollector}.
+ *
+ * @lucene.experimental
+ */
+abstract class AbstractRnnVectorQuery extends AbstractKnnVectorQuery {
+  private static final TopDocs NO_RESULTS = TopDocsCollector.EMPTY_TOPDOCS;

Review Comment:
   No strong opinions here, I had just kept it similar to `AbstractKnnVectorQuery`
   Changed now..



##########
lucene/core/src/java/org/apache/lucene/search/AbstractRnnVectorQuery.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.LeafReaderContext;
+
+/**
+ * Search for all (approximate) vectors within a radius using the {@link RnnCollector}.
+ *
+ * @lucene.experimental
+ */
+abstract class AbstractRnnVectorQuery extends AbstractKnnVectorQuery {
+  private static final TopDocs NO_RESULTS = TopDocsCollector.EMPTY_TOPDOCS;
+
+  protected final float traversalThreshold, resultThreshold;

Review Comment:
   Added



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


Re: [PR] Add support for radius-based vector searches [lucene]

Posted by "kaivalnp (via GitHub)" <gi...@apache.org>.
kaivalnp commented on PR #12679:
URL: https://github.com/apache/lucene/pull/12679#issuecomment-1768862106

   > the Collector is full by flagging "incomplete" (I think this is possible) once a threshold is reached
   
   Do you mean that we return incomplete results?
   
   Instead, maybe we can:
   1. Ask for a sane limit on the number of nodes to visit from the user
   2. If this limit is reached (possibly when the supplied `traversalThreshold` is too low), then we break out of HNSW search
   3. Now instead of performing a [greedy `#exactSearch`](https://github.com/kaivalnp/lucene/blob/radius-based-vector-search/lucene/core/src/java/org/apache/lucene/search/AbstractRnnVectorQuery.java#L53-L74) and collecting everything into a list, we return a `TwoPhaseIterator` where the [`#matches`](https://github.com/apache/lucene/blob/main/lucene/core/src/java/org/apache/lucene/search/TwoPhaseIterator.java#L112) call performs the underlying dot product comparison and returns `true` or `false` based on whether the computed score is above the `resultThreshold`
   4. This way, we can perform an "exact search" lazily, and only compute vector similarity on required documents (for example: if this query is a child of some `BooleanQuery`, then the actual number of documents for which we'll need to compute similarity is greatly reduced). The worst case will still be an exact search on all documents
   
   This "lazy-loading" works very well for our use case because the fact that a vector matches our query or not is independent of other vectors (unlike in K-NN, where given a query and an arbitrary doc vector, we cannot say whether the doc vector will be in the `topK` results of the query)
   
   Is this what you had in mind earlier @jpountz?
   
   > I will try and replicate with Lucene Util.
   
   Yes, I took inspiration from [`KnnGraphTester`](https://github.com/mikemccand/luceneutil/blob/master/src/main/KnnGraphTester.java) to write a local benchmark, but may have made some silly mistakes. It'll be good to get an independent set of benchmark results..


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


Re: [PR] Add support for similarity-based vector searches [lucene]

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12679:
URL: https://github.com/apache/lucene/pull/12679#discussion_r1419656899


##########
lucene/CHANGES.txt:
##########
@@ -167,7 +167,10 @@ API Changes
 
 New Features
 ---------------------
-(No changes)
+
+* GITHUB#12679: Add support for similarity-based vector searches. Finds all vectors scoring above a `resultSimilarity`
+  while traversing the HNSW graph till better-scoring nodes are available, or the best candidate is below a score of
+  `traversalSimilarity` in the lowest level. (Aditya Prakash, Kaival Parikh)

Review Comment:
   Maybe add the vector names?



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


Re: [PR] Add support for similarity-based vector searches [lucene]

Posted by "kaivalnp (via GitHub)" <gi...@apache.org>.
kaivalnp commented on code in PR #12679:
URL: https://github.com/apache/lucene/pull/12679#discussion_r1419688833


##########
lucene/CHANGES.txt:
##########
@@ -167,7 +167,10 @@ API Changes
 
 New Features
 ---------------------
-(No changes)
+
+* GITHUB#12679: Add support for similarity-based vector searches. Finds all vectors scoring above a `resultSimilarity`
+  while traversing the HNSW graph till better-scoring nodes are available, or the best candidate is below a score of
+  `traversalSimilarity` in the lowest level. (Aditya Prakash, Kaival Parikh)

Review Comment:
   Ahh got it.. Updated now :)



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


Re: [PR] Add support for similarity-based vector searches [lucene]

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent merged PR #12679:
URL: https://github.com/apache/lucene/pull/12679


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


Re: [PR] Add support for similarity-based vector searches [lucene]

Posted by "kaivalnp (via GitHub)" <gi...@apache.org>.
kaivalnp commented on PR #12679:
URL: https://github.com/apache/lucene/pull/12679#issuecomment-1780186180

   Thank you! I'll try to incorporate earlier suggestions in the meanwhile


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