jimczi commented on code in PR #12962:
URL: https://github.com/apache/lucene/pull/12962#discussion_r1474802862


##########
lucene/core/src/java/org/apache/lucene/index/LeafReader.java:
##########
@@ -236,27 +235,24 @@ public final PostingsEnum postings(Term term) throws 
IOException {
    *
    * @param field the vector field to search
    * @param target the vector-valued query
-   * @param k the number of docs to return
    * @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
+   * @param knnCollector collector with settings for gathering the vector 
results.
    * @return the k nearest neighbor documents, along with their 
(searchStrategy-specific) scores.
    * @lucene.experimental
    */
   public final TopDocs searchNearestVectors(
-      String field, float[] target, int k, Bits acceptDocs, int visitedLimit) 
throws IOException {
+      String field, float[] target, Bits acceptDocs, KnnCollector 
knnCollector) throws IOException {

Review Comment:
   Is it a leftover? There's already a variant with the `KnnCollector` in the 
abstract functions. I don't think we should change this signature, it's ok if 
this version doesn't use the global queue imo.



##########
lucene/core/src/java/org/apache/lucene/search/AbstractKnnVectorQuery.java:
##########
@@ -79,24 +83,32 @@ public Query rewrite(IndexSearcher indexSearcher) throws 
IOException {
       filterWeight = null;
     }
 
+    final boolean supportsConcurrency = indexSearcher.getSlices().length > 1;
+    KnnCollectorManager<?> knnCollectorManager = getKnnCollectorManager(k, 
supportsConcurrency);
+
     TaskExecutor taskExecutor = indexSearcher.getTaskExecutor();
     List<LeafReaderContext> leafReaderContexts = reader.leaves();
     List<Callable<TopDocs>> tasks = new ArrayList<>(leafReaderContexts.size());
     for (LeafReaderContext context : leafReaderContexts) {
-      tasks.add(() -> searchLeaf(context, filterWeight));
+      tasks.add(() -> searchLeaf(context, filterWeight, knnCollectorManager));
     }
     TopDocs[] perLeafResults = 
taskExecutor.invokeAll(tasks).toArray(TopDocs[]::new);
 
     // Merge sort the results
     TopDocs topK = mergeLeafResults(perLeafResults);
+    if (infoStream.isEnabled("KnnVectorQuery")) {
+      infoStream.message("KnnVectorQuery", "visited:" + topK.totalHits.value);

Review Comment:
   looks like a leftover too?



##########
lucene/core/src/java/org/apache/lucene/search/AbstractKnnVectorQuery.java:
##########
@@ -79,24 +83,32 @@ public Query rewrite(IndexSearcher indexSearcher) throws 
IOException {
       filterWeight = null;
     }
 
+    final boolean supportsConcurrency = indexSearcher.getSlices().length > 1;

Review Comment:
   A slice can have multiple segments so it should rather check the reader's 
leaves here and maybe call the boolean `isMultiSegments`?



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