You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by "benwtrent (via GitHub)" <gi...@apache.org> on 2023/02/28 16:11:56 UTC

[GitHub] [lucene] benwtrent commented on a diff in pull request #12160: Concurrent rewrite for KnnVectorQuery

benwtrent commented on code in PR #12160:
URL: https://github.com/apache/lucene/pull/12160#discussion_r1120326066


##########
lucene/core/src/java/org/apache/lucene/search/AbstractKnnVectorQuery.java:
##########
@@ -73,17 +77,48 @@ public Query rewrite(IndexSearcher indexSearcher) throws IOException {
               .build();
       Query rewritten = indexSearcher.rewrite(booleanQuery);
       filterWeight = indexSearcher.createWeight(rewritten, ScoreMode.COMPLETE_NO_SCORES, 1f);
+    } else {
+      filterWeight = null;
     }
 
-    for (LeafReaderContext ctx : reader.leaves()) {
-      TopDocs results = searchLeaf(ctx, filterWeight);
-      if (ctx.docBase > 0) {
-        for (ScoreDoc scoreDoc : results.scoreDocs) {
-          scoreDoc.doc += ctx.docBase;
-        }
-      }
-      perLeafResults[ctx.ord] = results;
-    }
+    List<FutureTask<TopDocs>> tasks =
+        reader.leaves().stream()
+            .map(
+                ctx ->
+                    new FutureTask<>(
+                        () -> {
+                          try {
+                            TopDocs results = searchLeaf(ctx, filterWeight);
+                            if (ctx.docBase > 0) {
+                              for (ScoreDoc scoreDoc : results.scoreDocs) {
+                                scoreDoc.doc += ctx.docBase;
+                              }
+                            }
+                            return results;
+                          } catch (IOException e) {
+                            throw new UncheckedIOException(e);
+                          }
+                        }))
+            .toList();
+
+    Executor executor = Objects.requireNonNullElse(indexSearcher.getExecutor(), Runnable::run);
+    SliceExecutor sliceExecutor = new SliceExecutor(executor);
+    sliceExecutor.invokeAll(tasks);

Review Comment:
   I really like this change, but it seems to be we have a simple optimization opportunity here.
   
   We shouldn't bother with any parallelism if `indexSearcher.getExecutor() == null || reader.leaves().size() <= 1`. Its a simple `if` branch that allows us to remove all the overhead associated with parallel rewrites when no parallelism can be achieved. 



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