jpountz commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1274892128


##########
lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsReader.java:
##########
@@ -278,7 +276,7 @@ public TopDocs search(String field, float[] target, int k, 
Bits acceptDocs, int
   }
 
   @Override
-  public TopDocs search(String field, byte[] target, int k, Bits acceptDocs, 
int visitedLimit)
+  public TopDocs search(String field, byte[] target, KnnResults knnResults, 
Bits acceptDocs)

Review Comment:
   I'm not clear if this is a temporary thing and you plan on collecting 
vectors into the `KnnResults` object in a follow-up commit, or if this is the 
way things will be. I don't like that this is bypassing the `KnnResults` 
object, e.g. passing a `ToParentJoinKnnResults` would still return child doc 
IDs instead of parent doc IDs? I'm assuming it's fixable, but maybe this old 
codec makes it challenging?



##########
lucene/core/src/java/org/apache/lucene/index/LeafReader.java:
##########
@@ -268,8 +272,68 @@ public abstract TopDocs searchNearestVectors(
    * @return the k nearest neighbor documents, along with their 
(searchStrategy-specific) scores.
    * @lucene.experimental
    */
+  public final TopDocs searchNearestVectors(
+      String field, byte[] target, int k, Bits acceptDocs, int visitedLimit) 
throws IOException {
+    return searchNearestVectors(field, target, new TopKnnResults(k, 
visitedLimit), acceptDocs);
+  }
+
+  /**
+   * Return the k nearest neighbor documents as determined by comparison of 
their vector values for
+   * this field, to the given vector, by the field's similarity function. The 
score of each document
+   * is derived from the vector similarity in a way that ensures scores are 
positive and that a
+   * larger score corresponds to a higher ranking.
+   *
+   * <p>The search is allowed to be approximate, meaning the results are not 
guaranteed to be the
+   * true k closest neighbors. For large values of k (for example when k is 
close to the total
+   * number of documents), the search may also retrieve fewer than k documents.
+   *
+   * <p>The returned {@link TopDocs} will contain a {@link ScoreDoc} for each 
nearest neighbor, in
+   * order of their similarity to the query vector (decreasing scores). The 
{@link TotalHits}
+   * contains the number of documents visited during the search. If the search 
stopped early because
+   * it hit {@code visitedLimit}, it is indicated through the relation {@code
+   * TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO}.
+   *
+   * <p>The behavior is undefined if the given field doesn't have KNN vectors 
enabled on its {@link
+   * FieldInfo}. The return value is never {@code null}.
+   *
+   * @param field the vector field to search
+   * @param target the vector-valued query
+   * @param knnResults collector and topK for gathering the vector results
+   * @param acceptDocs {@link Bits} that represents the allowed documents to 
match, or {@code null}
+   *     if they are all allowed to match.
+   * @return the k nearest neighbor documents, along with their 
(similarity-specific) scores.
+   */
+  public abstract TopDocs searchNearestVectors(

Review Comment:
   should it return `void` as well and require callers to pull top docs on the 
`KnnResults` object?



##########
lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsReader.java:
##########
@@ -79,14 +80,13 @@ protected KnnVectorsReader() {}
    *
    * @param field the vector field to search
    * @param target the vector-valued query
-   * @param k the number of docs to return
+   * @param knnResults a KnnResults collector and relevant settings for 
gathering vector results
    * @param acceptDocs {@link Bits} that represents the allowed documents to 
match, or {@code null}
    *     if they are all allowed to match.
-   * @param visitedLimit the maximum number of nodes that the search is 
allowed to visit
    * @return the k nearest neighbor documents, along with their 
(similarity-specific) scores.
    */
   public abstract TopDocs search(
-      String field, float[] target, int k, Bits acceptDocs, int visitedLimit) 
throws IOException;
+      String field, float[] target, KnnResults knnResults, Bits acceptDocs) 
throws IOException;

Review Comment:
   Should these methods return `void`, and make callers responsible for pulling 
top docs from the `KnnResults` object?



##########
lucene/core/src/java/org/apache/lucene/search/KnnResults.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;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs 
from the gathered

Review Comment:
   I'm tempted to rename it to KnnCollector given this description. This would 
make sense to me as this is very similar to query collectors?



##########
lucene/core/src/java/org/apache/lucene/index/LeafReader.java:
##########
@@ -268,8 +272,68 @@ public abstract TopDocs searchNearestVectors(
    * @return the k nearest neighbor documents, along with their 
(searchStrategy-specific) scores.
    * @lucene.experimental
    */
+  public final TopDocs searchNearestVectors(
+      String field, byte[] target, int k, Bits acceptDocs, int visitedLimit) 
throws IOException {
+    return searchNearestVectors(field, target, new TopKnnResults(k, 
visitedLimit), acceptDocs);
+  }
+
+  /**
+   * Return the k nearest neighbor documents as determined by comparison of 
their vector values for
+   * this field, to the given vector, by the field's similarity function. The 
score of each document
+   * is derived from the vector similarity in a way that ensures scores are 
positive and that a
+   * larger score corresponds to a higher ranking.
+   *
+   * <p>The search is allowed to be approximate, meaning the results are not 
guaranteed to be the
+   * true k closest neighbors. For large values of k (for example when k is 
close to the total
+   * number of documents), the search may also retrieve fewer than k documents.
+   *
+   * <p>The returned {@link TopDocs} will contain a {@link ScoreDoc} for each 
nearest neighbor, in
+   * order of their similarity to the query vector (decreasing scores). The 
{@link TotalHits}
+   * contains the number of documents visited during the search. If the search 
stopped early because
+   * it hit {@code visitedLimit}, it is indicated through the relation {@code
+   * TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO}.
+   *
+   * <p>The behavior is undefined if the given field doesn't have KNN vectors 
enabled on its {@link
+   * FieldInfo}. The return value is never {@code null}.
+   *
+   * @param field the vector field to search
+   * @param target the vector-valued query
+   * @param knnResults collector and topK for gathering the vector results

Review Comment:
   maybe add something about the fact that these objects should not be reused 
across calls to `search`



##########
lucene/core/src/java/org/apache/lucene/search/KnnResults.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;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs 
from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  protected long visitedCount;

Review Comment:
   should it be private? sub-classes already have visitedCount() if they need 
to access the value?



##########
lucene/core/src/java/org/apache/lucene/search/KnnResults.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;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs 
from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  protected long visitedCount;
+  private final long visitLimit;
+  private final int k;
+
+  protected KnnResults(int k, long visitLimit) {
+    this.visitLimit = visitLimit;
+    this.k = k;
+  }
+
+  /**
+   * If search visits too many documents, the results collector will terminate 
early. Usually, this
+   * is due to some restricted filter on the document set.
+   *
+   * <p>When collection is earlyTerminated, the results are not a correct 
representation of k
+   * nearest neighbors.
+   *
+   * @return is the current result set marked as incomplete?
+   */
+  public final boolean earlyTerminated() {
+    return visitedCount >= visitLimit;
+  }
+
+  /**
+   * @param count increments the visited vector count, must be greater than 0.
+   */
+  public final void incVisitedCount(int count) {
+    assert count > 0;
+    this.visitedCount += count;
+  }

Review Comment:
   I wonder if we should keep all methods abstract in this base implementation. 
When we had default implementations like that in the past, it then made it hard 
to wrap these classes as you couldn't delegate `incVisitedCount` to the inner 
`KnnResults`. Maybe split it into `KnnResults` that defines the API and 
`AbstractKnnResults` that includes this base implementation?



##########
lucene/core/src/java/org/apache/lucene/search/KnnResults.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;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs 
from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  protected long visitedCount;
+  private final long visitLimit;
+  private final int k;
+
+  protected KnnResults(int k, long visitLimit) {
+    this.visitLimit = visitLimit;
+    this.k = k;
+  }
+
+  /**
+   * If search visits too many documents, the results collector will terminate 
early. Usually, this
+   * is due to some restricted filter on the document set.
+   *
+   * <p>When collection is earlyTerminated, the results are not a correct 
representation of k
+   * nearest neighbors.
+   *
+   * @return is the current result set marked as incomplete?
+   */
+  public final boolean earlyTerminated() {
+    return visitedCount >= visitLimit;
+  }
+
+  /**
+   * @param count increments the visited vector count, must be greater than 0.
+   */
+  public final void incVisitedCount(int count) {
+    assert count > 0;
+    this.visitedCount += count;
+  }
+
+  /**
+   * @return the current visited vector count
+   */
+  public final long visitedCount() {
+    return visitedCount;
+  }
+
+  /**
+   * @return the visited vector limit
+   */
+  public final long visitLimit() {
+    return visitLimit;
+  }
+
+  /**
+   * @return the expected number of collected results
+   */
+  public final int k() {
+    return k;
+  }
+
+  /**
+   * Collect the provided docId and include in the result set.
+   *
+   * @param docId of the vector to collect
+   * @param similarity its calculated similarity
+   * @return true if the vector is collected

Review Comment:
   Maybe clarify what it means for a vector to be collected. My understanding 
is that the semantics we want are that if this returns `false` then 
`minCompetitiveSimilarity` is guaranteed not to have been updated. Thinking out 
loud: I wonder how much perf this gives, maybe this could return `void` and 
we'd pull `minCompetitiveSimilarity` all the time in the search logic.



##########
lucene/join/src/java/org/apache/lucene/search/join/ToParentJoinKnnResults.java:
##########
@@ -0,0 +1,296 @@
+/*
+ * 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.join;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.lucene.search.KnnResults;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BitSet;
+
+/** parent joining knn results, vectorIds are deduplicated according to the 
parent bit set. */
+public class ToParentJoinKnnResults extends KnnResults {

Review Comment:
   This class could be pkg-private? (or tagged with `@lucene.internal` if you'd 
like to be able to reuse it elsewhere)



##########
lucene/core/src/java/org/apache/lucene/util/hnsw/TopKnnResults.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.util.hnsw;
+
+import org.apache.lucene.search.KnnResults;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+
+/**
+ * TopKnnResults is a specific KnnResults. A minHeap is used to keep track of 
the currently
+ * collected vectors allowing for efficient updates as better vectors are 
collected.
+ */
+public class TopKnnResults extends KnnResults {
+
+  protected final NeighborQueue queue;
+
+  /**
+   * @param k the number of neighbors to collect
+   * @param visitLimit how many vector nodes the results are allowed to visit
+   */
+  public TopKnnResults(int k, int visitLimit) {
+    super(k, visitLimit);
+    this.queue = new NeighborQueue(k, false);
+  }
+
+  @Override
+  public boolean collect(int docId, float similarity) {
+    return queue.insertWithOverflow(docId, similarity);
+  }
+
+  @Override
+  public float minCompetitiveSimilarity() {
+    return queue.size() >= k() ? queue.topScore() : Float.NEGATIVE_INFINITY;
+  }
+
+  @Override
+  public TopDocs topDocs() {
+    while (queue.size() > k()) {
+      queue.pop();
+    }
+    int i = 0;
+    ScoreDoc[] scoreDocs = new ScoreDoc[queue.size()];
+    while (i < scoreDocs.length) {
+      int node = queue.topNode();
+      float score = queue.topScore();
+      queue.pop();
+      scoreDocs[scoreDocs.length - ++i] = new ScoreDoc(node, score);
+    }
+    TotalHits.Relation relation =
+        earlyTerminated()
+            ? TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO
+            : TotalHits.Relation.EQUAL_TO;
+    return new TopDocs(new TotalHits(visitedCount(), relation), scoreDocs);
+  }
+
+  @Override
+  public String toString() {
+    return "TopKnnResults[" + queue.size() + "]";

Review Comment:
   Without reading the code, I would have assumed the number to be the `k` 
rather than the size of the queue.
   
   ```suggestion
       return "TopKnnResults[k=" + k() + ", size=" + queue.size() + "]";
   ```



-- 
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-unsubscr...@lucene.apache.org

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


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

Reply via email to