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/02/19 23:21:01 UTC

[GitHub] [lucene] kaivalnp opened a new pull request, #12160: Concurrent rewrite for KnnVectorQuery

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

   ### Description
   Issue #11862
   
   ### Solution
   `AbstractKnnVectorQuery` currently performs HNSW searches (one per-segment) iteratively
   Since this is done in a single thread, we can make it concurrent by spinning off per-segment searches to different threads (and make use of available processors)
   
   The actual search is performed in `Query#rewrite`, and support to allow concurrency there was added recently (#11838) by passing an `IndexSearcher` (which wraps an `IndexReader` and `Executor`)
   
   Proposing to achieve this by `CompletableFuture`:
   - If the `Executor` is not set, we can perform a blocking call to `CompletableFuture#completedFuture`
   - Else we submit the task of per-segment search to the `Executor` using `CompletableFuture#supplyAsync`


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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


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


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

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


##########
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:
   > That said, your suggestion aligns with how IndexSearcher currently works, so maybe we should apply it for now and discuss in a follow-up issue whether we should also delegate to the executor when there is a single segment.
   
   I am fine with that. 
   
   I think also that we could only check `indexSearcher.getExecutor() == null` instead of making a decision for the caller regarding the number of leaves.
   
   So, I would say for now only check if `indexSearcher.getExecutor() == null` and if it is, do it the old way.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] zhaih commented on pull request #12160: Concurrent rewrite for KnnVectorQuery

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

   I think the concurrent rewrite should act the same way(one thread per
   slice) ideally. Do you want to open a PR on that? Or open an issue and I
   can probably work on it recently
   
   On Tue, May 23, 2023, 13:42 Luca Cavanna ***@***.***> wrote:
   
   > Question on this change: index searcher parallelizes search over leaf
   > slices, while the concurrent rewrite for knn vector query will request one
   > thread per segment. Is this on purpose or should the concurrent rewrite
   > also leverage the slices for its parallelism?
   >
   > —
   > Reply to this email directly, view it on GitHub
   > <https://github.com/apache/lucene/pull/12160#issuecomment-1560097486>, or
   > unsubscribe
   > <https://github.com/notifications/unsubscribe-auth/AEFSB7AKDU5Z6CZPHOGLLPDXHUOLTANCNFSM6AAAAAAVBHNVRE>
   > .
   > You are receiving this because you modified the open/close state.Message
   > ID: ***@***.***>
   >
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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


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


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

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


##########
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:
   @kaivalnp could you (instead of using `Runnable::run`) just do the regular loop as it was previously if `indexSearcher.getExecutor() == null`?
   
   If `getExecutor()` is not null, we should assume the caller wants it used. @jpountz is correct 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


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

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


##########
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:
   Thanks for the input!
   This was really helpful in reducing overhead for non-concurrent search, and improving readability!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] javanna commented on pull request #12160: Concurrent rewrite for KnnVectorQuery

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

   Question on this change: index searcher parallelizes search over leaf slices, while the concurrent rewrite for knn vector query will request one thread per segment. Is this on purpose or should the concurrent rewrite also leverage the slices for its parallelism?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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


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


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

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


##########
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);
+
+    TopDocs[] perLeafResults =
+        tasks.stream()
+            .map(
+                task -> {
+                  try {
+                    return task.get();
+                  } catch (ExecutionException e) {
+                    throw new RuntimeException(e.getCause());
+                  } catch (InterruptedException e) {
+                    throw new RuntimeException(e);

Review Comment:
   Makes sense! This is more suitable



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] mkhludnev commented on pull request #12160: Concurrent rewrite for KnnVectorQuery

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

   May this failure relevant to this 
   https://jenkins.thetaphi.de/job/Lucene-9.x-Linux/8927/testReport/junit/org.apache.lucene.search/TestKnnFloatVectorQuery/testDocAndScoreQueryBasics/ 
   ```
   java.lang.IllegalStateException: This DocAndScore query was created by a different reader
   	at __randomizedtesting.SeedInfo.seed([125C1E79EB05AAC5:3BA19DF38099A68]:0)
   	at org.apache.lucene.search.AbstractKnnVectorQuery$DocAndScoreQuery.createWeight(AbstractKnnVectorQuery.java:298)
   	at org.apache.lucene.search.TestKnnFloatVectorQuery.testDocAndScoreQueryBasics(TestKnnFloatVectorQuery.java:204)
   ```
   ??


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] zhaih commented on pull request #12160: Concurrent rewrite for KnnVectorQuery

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

   > Do you mind then guiding this PR through the merging and potential backporting process?
   
   Yeah I can help with that, I would suggest let's keep it in Lucene 10 for now, and if the previous PR is able to be backported to 9x in the future, I'll remember to backport this PR as well (along with the CHANGES.txt entry moving to appropriate place)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] kaivalnp commented on pull request #12160: Concurrent rewrite for KnnVectorQuery

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

   I ran some benchmarks as well (Columns 2-4 are latency in ms):
   
   ### enwiki (topK = 100, segment count = 10)
   | recall | Sequential | CompletableFuture | SliceExecutor | nDoc | fanout | maxConn | beamWidth |
   |-|-|-|-|-|-|-|-|
   | 0.995 |  1.00 |  0.31 |  0.32 |   10000 |  0 | 16 | 32 |
   | 0.998 |  1.33 |  0.37 |  0.38 |   10000 | 50 | 16 | 32 |
   | 0.998 |  1.09 |  0.31 |  0.32 |   10000 |  0 | 16 | 64 |
   | 0.999 |  1.43 |  0.37 |  0.40 |   10000 | 50 | 16 | 64 |
   | 0.995 |  0.99 |  0.29 |  0.31 |   10000 |  0 | 32 | 32 |
   | 0.998 |  1.31 |  0.35 |  0.37 |   10000 | 50 | 32 | 32 |
   | 0.998 |  1.05 |  0.29 |  0.31 |   10000 |  0 | 32 | 64 |
   | 0.999 |  1.40 |  0.36 |  0.38 |   10000 | 50 | 32 | 64 |
   | 0.987 |  1.73 |  0.39 |  0.41 |  100000 |  0 | 16 | 32 |
   | 0.992 |  2.37 |  0.49 |  0.52 |  100000 | 50 | 16 | 32 |
   | 0.993 |  1.92 |  0.41 |  0.44 |  100000 |  0 | 16 | 64 |
   | 0.996 |  2.57 |  0.53 |  0.56 |  100000 | 50 | 16 | 64 |
   | 0.987 |  1.74 |  0.39 |  0.41 |  100000 |  0 | 32 | 32 |
   | 0.992 |  2.34 |  0.50 |  0.52 |  100000 | 50 | 32 | 32 |
   | 0.994 |  1.98 |  0.42 |  0.45 |  100000 |  0 | 32 | 64 |
   | 0.997 |  2.68 |  0.54 |  0.57 |  100000 | 50 | 32 | 64 |
   | 0.971 |  2.76 |  0.56 |  0.46 | 1000000 |  0 | 16 | 32 |
   | 0.982 |  3.75 |  0.72 |  0.60 | 1000000 | 50 | 16 | 32 |
   | 0.985 |  3.26 |  0.61 |  0.55 | 1000000 |  0 | 16 | 64 |
   | 0.991 |  4.24 |  0.80 |  0.64 | 1000000 | 50 | 16 | 64 |
   | 0.973 |  2.80 |  0.58 |  0.51 | 1000000 |  0 | 32 | 32 |
   | 0.983 |  3.88 |  0.75 |  0.62 | 1000000 | 50 | 32 | 32 |
   | 0.986 |  3.33 |  0.65 |  0.57 | 1000000 |  0 | 32 | 64 |
   | 0.992 |  4.57 |  0.85 |  0.70 | 1000000 | 50 | 32 | 64 |
   
   ### enwiki (topK = 100, segment count = 5)
   | recall | Sequential | CompletableFuture | SliceExecutor | nDoc | fanout | maxConn | beamWidth |
   |-|-|-|-|-|-|-|-|
   | 0.991 |  0.59 |  0.28 |  0.23 |   10000 |  0 | 16 | 32 |
   | 0.996 |  0.84 |  0.34 |  0.27 |   10000 | 50 | 16 | 32 |
   | 0.997 |  0.62 |  0.28 |  0.23 |   10000 |  0 | 16 | 64 |
   | 0.999 |  0.89 |  0.35 |  0.31 |   10000 | 50 | 16 | 64 |
   | 0.991 |  0.60 |  0.26 |  0.21 |   10000 |  0 | 32 | 32 |
   | 0.995 |  0.81 |  0.35 |  0.26 |   10000 | 50 | 32 | 32 |
   | 0.997 |  0.65 |  0.27 |  0.22 |   10000 |  0 | 32 | 64 |
   | 0.999 |  0.86 |  0.36 |  0.29 |   10000 | 50 | 32 | 64 |
   | 0.978 |  0.97 |  0.36 |  0.29 |  100000 |  0 | 16 | 32 |
   | 0.987 |  1.29 |  0.47 |  0.38 |  100000 | 50 | 16 | 32 |
   | 0.989 |  1.08 |  0.40 |  0.31 |  100000 |  0 | 16 | 64 |
   | 0.994 |  1.46 |  0.52 |  0.41 |  100000 | 50 | 16 | 64 |
   | 0.977 |  0.98 |  0.36 |  0.31 |  100000 |  0 | 32 | 32 |
   | 0.987 |  1.32 |  0.49 |  0.37 |  100000 | 50 | 32 | 32 |
   | 0.989 |  1.14 |  0.40 |  0.34 |  100000 |  0 | 32 | 64 |
   | 0.994 |  1.55 |  0.56 |  0.42 |  100000 | 50 | 32 | 64 |
   | 0.957 |  1.54 |  0.57 |  0.45 | 1000000 |  0 | 16 | 32 |
   | 0.972 |  2.07 |  0.70 |  0.60 | 1000000 | 50 | 16 | 32 |
   | 0.976 |  1.71 |  0.61 |  0.53 | 1000000 |  0 | 16 | 64 |
   | 0.985 |  2.33 |  0.76 |  0.65 | 1000000 | 50 | 16 | 64 |
   | 0.959 |  1.58 |  0.57 |  0.45 | 1000000 |  0 | 32 | 32 |
   | 0.974 |  2.17 |  0.73 |  0.63 | 1000000 | 50 | 32 | 32 |
   | 0.978 |  1.88 |  0.65 |  0.53 | 1000000 |  0 | 32 | 64 |
   | 0.987 |  2.57 |  0.83 |  0.69 | 1000000 | 50 | 32 | 64 |
   
   ### enwiki (topK = 100, segment count = 1)
   | recall | Sequential | CompletableFuture | SliceExecutor | nDoc | fanout | maxConn | beamWidth |
   |-|-|-|-|-|-|-|-|
   | 0.941 |  0.23 |  0.25 |  0.21 |   10000 |  0 | 16 | 32 |
   | 0.970 |  0.24 |  0.27 |  0.25 |   10000 | 50 | 16 | 32 |
   | 0.965 |  0.19 |  0.23 |  0.20 |   10000 |  0 | 16 | 64 |
   | 0.984 |  0.27 |  0.28 |  0.28 |   10000 | 50 | 16 | 64 |
   | 0.941 |  0.17 |  0.20 |  0.17 |   10000 |  0 | 32 | 32 |
   | 0.970 |  0.23 |  0.27 |  0.24 |   10000 | 50 | 32 | 32 |
   | 0.966 |  0.20 |  0.22 |  0.20 |   10000 |  0 | 32 | 64 |
   | 0.985 |  0.26 |  0.29 |  0.28 |   10000 | 50 | 32 | 64 |
   | 0.909 |  0.26 |  0.29 |  0.28 |  100000 |  0 | 16 | 32 |
   | 0.945 |  0.36 |  0.38 |  0.38 |  100000 | 50 | 16 | 32 |
   | 0.944 |  0.30 |  0.32 |  0.31 |  100000 |  0 | 16 | 64 |
   | 0.969 |  0.42 |  0.43 |  0.43 |  100000 | 50 | 16 | 64 |
   | 0.914 |  0.28 |  0.30 |  0.29 |  100000 |  0 | 32 | 32 |
   | 0.948 |  0.37 |  0.43 |  0.44 |  100000 | 50 | 32 | 32 |
   | 0.949 |  0.31 |  0.35 |  0.31 |  100000 |  0 | 32 | 64 |
   | 0.972 |  0.41 |  0.48 |  0.43 |  100000 | 50 | 32 | 64 |
   | 0.870 |  0.35 |  0.38 |  0.36 | 1000000 |  0 | 16 | 32 |
   | 0.911 |  0.48 |  0.51 |  0.50 | 1000000 | 50 | 16 | 32 |
   | 0.913 |  0.40 |  0.43 |  0.42 | 1000000 |  0 | 16 | 64 |
   | 0.945 |  0.55 |  0.59 |  0.57 | 1000000 | 50 | 16 | 64 |
   | 0.881 |  0.38 |  0.43 |  0.39 | 1000000 |  0 | 32 | 32 |
   | 0.919 |  0.52 |  0.57 |  0.55 | 1000000 | 50 | 32 | 32 |
   | 0.923 |  0.45 |  0.50 |  0.47 | 1000000 |  0 | 32 | 64 |
   | 0.954 |  0.62 |  0.67 |  0.65 | 1000000 | 50 | 32 | 64 |
   
   I used `KnnGraphTester` for this benchmark with two changes:
   - Used `elapsed` time instead of `totalCpuTime` (which measures time for which the main thread was running). This is important because the main thread may be "waiting" on executor threads to finish, and not count towards overall latency
   - Added an executor: `Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors())` to the `IndexSearcher` object, to "make use" of concurrency
   
   Please let me know if this sounds fine?
   
   As expected, the speedup is higher for larger number of segments since they are taken up in parallel. I have also tried to measure the overhead of searching from a single segment


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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


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


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

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


##########
lucene/core/src/java/org/apache/lucene/search/AbstractKnnVectorQuery.java:
##########
@@ -73,17 +76,41 @@ 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;
-    }
+    TopDocs[] perLeafResults =
+        reader.leaves().stream()
+            .map(
+                ctx -> {
+                  Supplier<TopDocs> supplier =
+                      () -> {
+                        try {
+                          TopDocs results = searchLeaf(ctx, filterWeight);
+                          if (ctx.docBase > 0) {
+                            for (ScoreDoc scoreDoc : results.scoreDocs) {
+                              scoreDoc.doc += ctx.docBase;
+                            }
+                          }
+                          return results;
+                        } catch (Exception e) {
+                          throw new CompletionException(e);
+                        }
+                      };
+
+                  Executor executor = indexSearcher.getExecutor();
+                  if (executor == null) {
+                    return CompletableFuture.completedFuture(supplier.get());
+                  } else {
+                    return CompletableFuture.supplyAsync(supplier, executor);

Review Comment:
   In `IndexSearcher` we're using [`SliceExecutor`](https://github.com/apache/lucene/blob/main/lucene/core/src/java/org/apache/lucene/search/SliceExecutor.java#L67) to make sure the main thread is also doing some work but not only wait for joining. 
   I think we can replicate the same logic here? (Since KNN search is likely to be slow so probably the main thread should do some work as well?)
   
   Maybe we can just use the `SliceExecutor` from `IndexSearcher` so that it might also kind of solving the load balancing problem?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] zhaih commented on pull request #12160: Concurrent rewrite for KnnVectorQuery

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

   @benwtrent I think the [concurrent rewrite API change](https://github.com/apache/lucene/pull/11840) is only at Lucene10 right now, it is a bit tricky to backport it to 9x and not yet done as discussed in the previous PR I linked.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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


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


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

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


##########
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 wonder if this is right. Thinking out loud, I would assume that users who leverage IndexSearcher concurrency generally have two thread pools, one that they pass to the IndexSearcher constructor that they expect to do the heavy work, and another one, which is the one where `IndexSearcher#search` is called, that mostly handles coordination and lightweight work such as merging top hits coming from different shards but generally spends most of its time waiting for work to complete in the other threadpool. Your optimization suggestion boils dow to running some heavy work (a vector search) in the coordinating threadpool when there is a single segment. If heavy work may happen in either threadpool, this makes sizing these threadpools complicated, as either you allocate `num_cores` threads to the threadpool that does heavy work but then you may end up with more than `num_cores` threads doing heavy work because some heavy work also happens in the coordinating threadpool., or you allocate
  less than `num_cores` threads but then you might not use all your hardware?
   
   That said, your suggestion aligns with how IndexSearcher currently works, so maybe we should apply it for now and discuss in a follow-up issue whether we should also delegate to the executor when there is a single segment.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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


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


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

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


##########
lucene/core/src/test/org/apache/lucene/search/BaseKnnVectorQueryTestCase.java:
##########
@@ -210,7 +210,10 @@ public void testDimensionMismatch() throws IOException {
       IndexSearcher searcher = newSearcher(reader);
       AbstractKnnVectorQuery kvq = getKnnVectorQuery("field", new float[] {0}, 10);
       IllegalArgumentException e =
-          expectThrows(IllegalArgumentException.class, () -> searcher.search(kvq, 10));
+          expectThrows(
+              RuntimeException.class,
+              IllegalArgumentException.class,

Review Comment:
   Thank you, that makes sense, I had made a wrong assumption about what expectThrows does with two exception types!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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


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


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

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


##########
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:
   > 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.
   
   I would totally agree with you here! We shouldn't add overhead for non-concurrent executions
   If I understand correctly, you are suggesting to add an `if` block with the condition:
   - When it evaluates to `true`, we want to perform search [as before](https://github.com/apache/lucene/blob/main/lucene/core/src/java/org/apache/lucene/search/AbstractKnnVectorQuery.java#L78-L86)
   - `else` we perform the search [as per this PR](https://github.com/kaivalnp/lucene/blob/concurrent-knn/lucene/core/src/java/org/apache/lucene/search/AbstractKnnVectorQuery.java#L84-L120)
   
   I have tried to implement the changes [here](https://github.com/apache/lucene/compare/main...kaivalnp:lucene:concurrent-knn-reduce-overhead). I ran some benchmarks for these (with the executor as `null`):
   
   ### enwiki (topK = 100, segment count = 10, executor = null)
   | recall | Sequential | SliceExecutor | ReduceOverhead | nDoc | fanout | maxConn | beamWidth |
   |-|-|-|-|-|-|-|-|
   | 0.995 |  0.95 |  0.96 |  0.95 |   10000 |  0 | 16 | 32 |
   | 0.998 |  1.26 |  1.30 |  1.29 |   10000 | 50 | 16 | 32 |
   | 0.998 |  1.05 |  1.07 |  1.07 |   10000 |  0 | 16 | 64 |
   | 0.999 |  1.41 |  1.43 |  1.43 |   10000 | 50 | 16 | 64 |
   | 0.995 |  0.98 |  0.99 |  0.98 |   10000 |  0 | 32 | 32 |
   | 0.998 |  1.31 |  1.33 |  1.34 |   10000 | 50 | 32 | 32 |
   | 0.998 |  0.99 |  1.01 |  1.01 |   10000 |  0 | 32 | 64 |
   | 0.999 |  1.33 |  1.36 |  1.36 |   10000 | 50 | 32 | 64 |
   | 0.987 |  1.70 |  1.70 |  1.71 |  100000 |  0 | 16 | 32 |
   | 0.992 |  2.30 |  2.30 |  2.31 |  100000 | 50 | 16 | 32 |
   | 0.993 |  1.92 |  1.89 |  1.94 |  100000 |  0 | 16 | 64 |
   | 0.996 |  2.63 |  2.65 |  2.64 |  100000 | 50 | 16 | 64 |
   | 0.987 |  1.73 |  1.70 |  1.74 |  100000 |  0 | 32 | 32 |
   | 0.992 |  2.34 |  2.30 |  2.37 |  100000 | 50 | 32 | 32 |
   | 0.994 |  1.96 |  1.92 |  1.98 |  100000 |  0 | 32 | 64 |
   | 0.997 |  2.66 |  2.61 |  2.69 |  100000 | 50 | 32 | 64 |
   | 0.971 |  2.72 |  2.70 |  2.74 | 1000000 |  0 | 16 | 32 |
   | 0.982 |  3.77 |  3.79 |  3.78 | 1000000 | 50 | 16 | 32 |
   | 0.985 |  3.13 |  3.19 |  3.19 | 1000000 |  0 | 16 | 64 |
   | 0.991 |  4.34 |  4.37 |  4.36 | 1000000 | 50 | 16 | 64 |
   | 0.973 |  2.86 |  2.94 |  2.94 | 1000000 |  0 | 32 | 32 |
   | 0.983 |  3.94 |  3.98 |  3.97 | 1000000 | 50 | 32 | 32 |
   | 0.986 |  3.38 |  3.37 |  3.38 | 1000000 |  0 | 32 | 64 |
   | 0.992 |  4.63 |  4.66 |  4.67 | 1000000 | 50 | 32 | 64 |
   
   ### enwiki (topK = 100, segment count = 5, executor = null)
   | recall | Sequential | SliceExecutor | ReduceOverhead | nDoc | fanout | maxConn | beamWidth |
   |-|-|-|-|-|-|-|-|
   | 0.991 |  0.59 |  0.61 |  0.59 |   10000 |  0 | 16 | 32 |
   | 0.996 |  0.82 |  0.83 |  0.81 |   10000 | 50 | 16 | 32 |
   | 0.997 |  0.61 |  0.62 |  0.60 |   10000 |  0 | 16 | 64 |
   | 0.999 |  0.88 |  0.88 |  0.86 |   10000 | 50 | 16 | 64 |
   | 0.991 |  0.59 |  0.59 |  0.58 |   10000 |  0 | 32 | 32 |
   | 0.995 |  0.80 |  0.81 |  0.80 |   10000 | 50 | 32 | 32 |
   | 0.997 |  0.64 |  0.64 |  0.62 |   10000 |  0 | 32 | 64 |
   | 0.999 |  0.87 |  0.88 |  0.89 |   10000 | 50 | 32 | 64 |
   | 0.978 |  1.09 |  1.08 |  1.08 |  100000 |  0 | 16 | 32 |
   | 0.987 |  1.29 |  1.32 |  1.34 |  100000 | 50 | 16 | 32 |
   | 0.989 |  1.10 |  1.09 |  1.10 |  100000 |  0 | 16 | 64 |
   | 0.994 |  1.48 |  1.49 |  1.46 |  100000 | 50 | 16 | 64 |
   | 0.977 |  0.98 |  0.99 |  0.98 |  100000 |  0 | 32 | 32 |
   | 0.987 |  1.33 |  1.35 |  1.34 |  100000 | 50 | 32 | 32 |
   | 0.989 |  1.13 |  1.14 |  1.13 |  100000 |  0 | 32 | 64 |
   | 0.994 |  1.55 |  1.55 |  1.53 |  100000 | 50 | 32 | 64 |
   | 0.957 |  1.48 |  1.52 |  1.49 | 1000000 |  0 | 16 | 32 |
   | 0.972 |  2.03 |  2.08 |  2.04 | 1000000 | 50 | 16 | 32 |
   | 0.976 |  1.70 |  1.73 |  1.71 | 1000000 |  0 | 16 | 64 |
   | 0.985 |  2.42 |  2.45 |  2.47 | 1000000 | 50 | 16 | 64 |
   | 0.959 |  1.67 |  1.65 |  1.66 | 1000000 |  0 | 32 | 32 |
   | 0.974 |  2.13 |  2.15 |  2.16 | 1000000 | 50 | 32 | 32 |
   | 0.978 |  1.89 |  1.84 |  1.89 | 1000000 |  0 | 32 | 64 |
   | 0.987 |  2.52 |  2.53 |  2.55 | 1000000 | 50 | 32 | 64 |
   
   ### enwiki (topK = 100, segment count = 1, executor = null)
   | recall | Sequential | SliceExecutor | ReduceOverhead | nDoc | fanout | maxConn | beamWidth |
   |-|-|-|-|-|-|-|-|
   | 0.941 |  0.22 |  0.21 |  0.24 |   10000 |  0 | 16 | 32 |
   | 0.970 |  0.24 |  0.24 |  0.25 |   10000 | 50 | 16 | 32 |
   | 0.965 |  0.20 |  0.19 |  0.20 |   10000 |  0 | 16 | 64 |
   | 0.984 |  0.28 |  0.27 |  0.28 |   10000 | 50 | 16 | 64 |
   | 0.941 |  0.18 |  0.17 |  0.18 |   10000 |  0 | 32 | 32 |
   | 0.970 |  0.24 |  0.23 |  0.23 |   10000 | 50 | 32 | 32 |
   | 0.966 |  0.20 |  0.20 |  0.20 |   10000 |  0 | 32 | 64 |
   | 0.985 |  0.28 |  0.27 |  0.26 |   10000 | 50 | 32 | 64 |
   | 0.909 |  0.27 |  0.27 |  0.27 |  100000 |  0 | 16 | 32 |
   | 0.945 |  0.38 |  0.36 |  0.37 |  100000 | 50 | 16 | 32 |
   | 0.944 |  0.32 |  0.30 |  0.30 |  100000 |  0 | 16 | 64 |
   | 0.969 |  0.43 |  0.41 |  0.42 |  100000 | 50 | 16 | 64 |
   | 0.914 |  0.28 |  0.28 |  0.29 |  100000 |  0 | 32 | 32 |
   | 0.948 |  0.39 |  0.38 |  0.38 |  100000 | 50 | 32 | 32 |
   | 0.949 |  0.30 |  0.30 |  0.32 |  100000 |  0 | 32 | 64 |
   | 0.972 |  0.44 |  0.41 |  0.40 |  100000 | 50 | 32 | 64 |
   | 0.870 |  0.35 |  0.34 |  0.35 | 1000000 |  0 | 16 | 32 |
   | 0.911 |  0.49 |  0.48 |  0.47 | 1000000 | 50 | 16 | 32 |
   | 0.913 |  0.40 |  0.40 |  0.41 | 1000000 |  0 | 16 | 64 |
   | 0.945 |  0.55 |  0.55 |  0.56 | 1000000 | 50 | 16 | 64 |
   | 0.881 |  0.38 |  0.39 |  0.38 | 1000000 |  0 | 32 | 32 |
   | 0.919 |  0.52 |  0.52 |  0.52 | 1000000 | 50 | 32 | 32 |
   | 0.923 |  0.45 |  0.45 |  0.46 | 1000000 |  0 | 32 | 64 |
   | 0.954 |  0.62 |  0.62 |  0.61 | 1000000 | 50 | 32 | 64 |
   
   There are a few places where it gives some speedup, but this seems to be too low (Note that there is also some logic duplication [here](https://github.com/kaivalnp/lucene/blob/concurrent-knn-reduce-overhead/lucene/core/src/java/org/apache/lucene/search/AbstractKnnVectorQuery.java#L88-L93) and [here](https://github.com/kaivalnp/lucene/blob/concurrent-knn-reduce-overhead/lucene/core/src/java/org/apache/lucene/search/AbstractKnnVectorQuery.java#L103-L108), which we would want to avoid, maybe by wrapping it in a callable. I tried that out locally and it was performing similar to worse)
   
   In the absence of an executor, we are setting it to `Runnable::run`, which performs the same tasks sequentially. My guess would be that its overhead is much lower compared to the search tasks, and IMO the readability earlier outweighs the separate `if` block
   
   Please let me know what you feel / if you had something else in mind?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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


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


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

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

   I foresee backporting this to 9.6. Unless @kaivalnp @jpountz or @zhaih object.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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


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


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

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

   Thank you for the context @zhaih, Do you mind then guiding this PR through the merging and potential backporting process? 
   
   Backporting would indeed be blocked until your change can be backported at some point.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] javanna commented on pull request #12160: Concurrent rewrite for KnnVectorQuery

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

   I opened #12325 .


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] zhaih merged pull request #12160: Concurrent rewrite for KnnVectorQuery

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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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


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


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

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


##########
lucene/core/src/test/org/apache/lucene/search/BaseKnnVectorQueryTestCase.java:
##########
@@ -210,7 +210,10 @@ public void testDimensionMismatch() throws IOException {
       IndexSearcher searcher = newSearcher(reader);
       AbstractKnnVectorQuery kvq = getKnnVectorQuery("field", new float[] {0}, 10);
       IllegalArgumentException e =
-          expectThrows(IllegalArgumentException.class, () -> searcher.search(kvq, 10));
+          expectThrows(
+              RuntimeException.class,
+              IllegalArgumentException.class,

Review Comment:
   I wanted to preserve the original functionality of the testcase: Checking for illegal arguments
   If we only check for the outer class, it may be possible that some other exception was thrown inside (maybe `RuntimeException(NullPointerException)`), but the test still passed?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] jpountz commented on pull request #12160: Concurrent rewrite for KnnVectorQuery

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

   +1 to backporting


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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


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


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

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


##########
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:
   For the logic duplication, it just updated the doc ids (by adding `ctx.docBase` to get index-level doc ids): and I put it in a separate function
   
   > I think the minor if statement is worth it. It creates fewer objects and is a simpler function. It might be more readable if you broke the results gathering into individual private methods.
   
   [Here](https://github.com/apache/lucene/compare/main...kaivalnp:lucene:concurrent-knn-reduce-overhead) are the sample changes, please let me know if these look good: and I'll commit it in this PR
   
   Note that I had to wrap the sequential execution in a `try - catch`, and wrap exceptions in a `RuntimeException` for consistency with exceptions thrown during parallel execution (also to pass test cases)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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


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


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

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


##########
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:
   > Note that there is also some logic duplication 
   
   Duplication of logic right next to each other is fine (IMO). I would keep it simple and duplicate those 4 lines.
   
   I would also change the if statement to only be `if(executor == null)`. 
   
   I think the minor `if` statement is worth it. It creates fewer objects and is a simpler function. It might be more readable if you broke the results gathering into individual private methods.
   
   `TopDocs[] gatherPerLeafResults(List<LeafReaderContext>,Weight)`
   
   `TopDocs[] gatherPerLeafResults(List<LeafReaderContext>,Weight,Executor)`
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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


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


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

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
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


[GitHub] [lucene] kaivalnp commented on pull request #12160: Concurrent rewrite for KnnVectorQuery

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

   > May this failure relevant to this
   https://jenkins.thetaphi.de/job/Lucene-9.x-Linux/8927/testReport/junit/org.apache.lucene.search/TestKnnFloatVectorQuery/testDocAndScoreQueryBasics/
   
   I don't think so: The build seems to be for `branch_9x` ([link](https://jenkins.thetaphi.de/job/Lucene-9.x-Linux/8927/) to job information), which does not have this change
   The `Revision` of the build also [shows the latest commit](https://gitbox.apache.org/repos/asf?p=lucene.git;a=commit;h=5ff8ea4952a2d1469c54d6a1774cae472e991169) on that branch, where [this commit is not present](https://gitbox.apache.org/repos/asf?p=lucene.git;a=shortlog;h=refs/heads/branch_9x)
   
   Please let me know if I'm missing something?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] kaivalnp commented on pull request #12160: Concurrent rewrite for KnnVectorQuery

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

   As a follow up here: We can also dive into breaking the per-segment search into sub-tasks, and make use of work-stealing thread pools (if one thread completes search in a smaller graph, it can "steal" some work from another thread that is performing a larger search by picking up some of it's sub-tasks)
   
   One thing to note here would be that sub-tasks must be large enough so that overhead from sub-task creation, thread switching, etc. does not cause performance drops instead
   
   Any suggestions?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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


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


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

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


##########
lucene/core/src/java/org/apache/lucene/search/AbstractKnnVectorQuery.java:
##########
@@ -73,17 +76,41 @@ 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;
-    }
+    TopDocs[] perLeafResults =
+        reader.leaves().stream()
+            .map(
+                ctx -> {
+                  Supplier<TopDocs> supplier =
+                      () -> {
+                        try {
+                          TopDocs results = searchLeaf(ctx, filterWeight);
+                          if (ctx.docBase > 0) {
+                            for (ScoreDoc scoreDoc : results.scoreDocs) {
+                              scoreDoc.doc += ctx.docBase;
+                            }
+                          }
+                          return results;
+                        } catch (Exception e) {
+                          throw new CompletionException(e);
+                        }
+                      };
+
+                  Executor executor = indexSearcher.getExecutor();
+                  if (executor == null) {
+                    return CompletableFuture.completedFuture(supplier.get());
+                  } else {
+                    return CompletableFuture.supplyAsync(supplier, executor);

Review Comment:
   Thanks for the input! This was helpful in reducing latency from thread switching further



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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


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


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

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


##########
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);
+
+    TopDocs[] perLeafResults =
+        tasks.stream()
+            .map(
+                task -> {
+                  try {
+                    return task.get();
+                  } catch (ExecutionException e) {
+                    throw new RuntimeException(e.getCause());

Review Comment:
   Thank for pointing to that javadoc, it makes sense to me. Maybe we should consider doing the same in IndexSearcher where we also catch ExecutionException (in a follow-up).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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


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


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

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


##########
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:
   > For the logic duplication
   
   I wouldn't worry about that. That makes things even more difficult to reason about. I would much rather have a method that takes in the filter weight and leaf contexts and one that takes the same parameters but with an added Executor.
    
   One called where `indexSearcher.getExecutor() == null` and the other when the executor is provided.
   Two methods like this:
   ```
     private TopDocs[] gatherLeafResults(
         List<LeafReaderContext> leafReaderContexts, Weight filterWeight) throws IOException {
       TopDocs[] perLeafResults = new TopDocs[leafReaderContexts.size()];
       for (LeafReaderContext ctx : leafReaderContexts) {
         TopDocs results = searchLeaf(ctx, filterWeight);
         if (ctx.docBase > 0) {
           for (ScoreDoc scoreDoc : results.scoreDocs) {
             scoreDoc.doc += ctx.docBase;
           }
         }
         perLeafResults[ctx.ord] = results;
       }
       return perLeafResults;
     }
   
     private TopDocs[] gatherLeafResults(
         List<LeafReaderContext> leafReaderContexts, Weight filterWeight, Executor executor) {
       List<FutureTask<TopDocs>> tasks =
           leafReaderContexts.stream()
               .map(
                   ctx ->
                       new FutureTask<>(
                           () -> {
                             TopDocs results = searchLeaf(ctx, filterWeight);
                             if (ctx.docBase > 0) {
                               for (ScoreDoc scoreDoc : results.scoreDocs) {
                                 scoreDoc.doc += ctx.docBase;
                               }
                             }
                             return results;
                           }))
               .toList();
   
       SliceExecutor sliceExecutor = new SliceExecutor(executor);
       sliceExecutor.invokeAll(tasks);
   
       return tasks.stream()
           .map(
               task -> {
                 try {
                   return task.get();
                 } catch (ExecutionException e) {
                   throw new RuntimeException(e.getCause());
                 } catch (InterruptedException e) {
                   throw new ThreadInterruptedException(e);
                 }
               })
           .toArray(TopDocs[]::new);
     }
   ```
   
   



##########
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:
   > For the logic duplication
   
   I wouldn't worry about that. That makes things even more difficult to reason about. I would much rather have a method that takes in the filter weight and leaf contexts and one that takes the same parameters but with an added Executor.
    
   One called where `indexSearcher.getExecutor() == null` and the other when the executor is provided.
   Two methods like this:
   ```java
     private TopDocs[] gatherLeafResults(
         List<LeafReaderContext> leafReaderContexts, Weight filterWeight) throws IOException {
       TopDocs[] perLeafResults = new TopDocs[leafReaderContexts.size()];
       for (LeafReaderContext ctx : leafReaderContexts) {
         TopDocs results = searchLeaf(ctx, filterWeight);
         if (ctx.docBase > 0) {
           for (ScoreDoc scoreDoc : results.scoreDocs) {
             scoreDoc.doc += ctx.docBase;
           }
         }
         perLeafResults[ctx.ord] = results;
       }
       return perLeafResults;
     }
   
     private TopDocs[] gatherLeafResults(
         List<LeafReaderContext> leafReaderContexts, Weight filterWeight, Executor executor) {
       List<FutureTask<TopDocs>> tasks =
           leafReaderContexts.stream()
               .map(
                   ctx ->
                       new FutureTask<>(
                           () -> {
                             TopDocs results = searchLeaf(ctx, filterWeight);
                             if (ctx.docBase > 0) {
                               for (ScoreDoc scoreDoc : results.scoreDocs) {
                                 scoreDoc.doc += ctx.docBase;
                               }
                             }
                             return results;
                           }))
               .toList();
   
       SliceExecutor sliceExecutor = new SliceExecutor(executor);
       sliceExecutor.invokeAll(tasks);
   
       return tasks.stream()
           .map(
               task -> {
                 try {
                   return task.get();
                 } catch (ExecutionException e) {
                   throw new RuntimeException(e.getCause());
                 } catch (InterruptedException e) {
                   throw new ThreadInterruptedException(e);
                 }
               })
           .toArray(TopDocs[]::new);
     }
   ```
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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


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


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

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


##########
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:
   > 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.
   
   I would totally agree with you here! We shouldn't add overhead for non-concurrent executions
   If I understand correctly, you are suggesting to add an `if` block with the condition:
   - When it evaluates to `true`, we want to perform search [as before](https://github.com/apache/lucene/blob/main/lucene/core/src/java/org/apache/lucene/search/AbstractKnnVectorQuery.java#L78-L86)
   - `else` we perform the search [as per this PR](https://github.com/kaivalnp/lucene/blob/concurrent-knn/lucene/core/src/java/org/apache/lucene/search/AbstractKnnVectorQuery.java#L84-L120)
   
   I have tried to implement the changes [here](https://github.com/apache/lucene/compare/main...kaivalnp:lucene:concurrent-knn-reduce-overhead). I ran some benchmarks for these (with the executor as `null`):
   
   ### enwiki (topK = 100, segment count = 10, executor = null)
   | recall | Sequential | SliceExecutor | ReduceOverhead | nDoc | fanout | maxConn | beamWidth |
   |-|-|-|-|-|-|-|-|
   | 0.995 |  0.95 |  0.96 |  0.95 |   10000 |  0 | 16 | 32 |
   | 0.998 |  1.26 |  1.30 |  1.29 |   10000 | 50 | 16 | 32 |
   | 0.998 |  1.05 |  1.07 |  1.07 |   10000 |  0 | 16 | 64 |
   | 0.999 |  1.41 |  1.43 |  1.43 |   10000 | 50 | 16 | 64 |
   | 0.995 |  0.98 |  0.99 |  0.98 |   10000 |  0 | 32 | 32 |
   | 0.998 |  1.31 |  1.33 |  1.34 |   10000 | 50 | 32 | 32 |
   | 0.998 |  0.99 |  1.01 |  1.01 |   10000 |  0 | 32 | 64 |
   | 0.999 |  1.33 |  1.36 |  1.36 |   10000 | 50 | 32 | 64 |
   | 0.987 |  1.70 |  1.70 |  1.71 |  100000 |  0 | 16 | 32 |
   | 0.992 |  2.30 |  2.30 |  2.31 |  100000 | 50 | 16 | 32 |
   | 0.993 |  1.92 |  1.89 |  1.94 |  100000 |  0 | 16 | 64 |
   | 0.996 |  2.63 |  2.65 |  2.64 |  100000 | 50 | 16 | 64 |
   | 0.987 |  1.73 |  1.70 |  1.74 |  100000 |  0 | 32 | 32 |
   | 0.992 |  2.34 |  2.30 |  2.37 |  100000 | 50 | 32 | 32 |
   | 0.994 |  1.96 |  1.92 |  1.98 |  100000 |  0 | 32 | 64 |
   | 0.997 |  2.66 |  2.61 |  2.69 |  100000 | 50 | 32 | 64 |
   | 0.971 |  2.72 |  2.70 |  2.74 | 1000000 |  0 | 16 | 32 |
   | 0.982 |  3.77 |  3.79 |  3.78 | 1000000 | 50 | 16 | 32 |
   | 0.985 |  3.13 |  3.19 |  3.19 | 1000000 |  0 | 16 | 64 |
   | 0.991 |  4.34 |  4.37 |  4.36 | 1000000 | 50 | 16 | 64 |
   | 0.973 |  2.86 |  2.94 |  2.94 | 1000000 |  0 | 32 | 32 |
   | 0.983 |  3.94 |  3.98 |  3.97 | 1000000 | 50 | 32 | 32 |
   | 0.986 |  3.38 |  3.37 |  3.38 | 1000000 |  0 | 32 | 64 |
   | 0.992 |  4.63 |  4.66 |  4.67 | 1000000 | 50 | 32 | 64 |
   
   ### enwiki (topK = 100, segment count = 5, executor = null)
   | recall | Sequential | SliceExecutor | ReduceOverhead | nDoc | fanout | maxConn | beamWidth |
   |-|-|-|-|-|-|-|-|
   | 0.991 |  0.59 |  0.61 |  0.59 |   10000 |  0 | 16 | 32 |
   | 0.996 |  0.82 |  0.83 |  0.81 |   10000 | 50 | 16 | 32 |
   | 0.997 |  0.61 |  0.62 |  0.60 |   10000 |  0 | 16 | 64 |
   | 0.999 |  0.88 |  0.88 |  0.86 |   10000 | 50 | 16 | 64 |
   | 0.991 |  0.59 |  0.59 |  0.58 |   10000 |  0 | 32 | 32 |
   | 0.995 |  0.80 |  0.81 |  0.80 |   10000 | 50 | 32 | 32 |
   | 0.997 |  0.64 |  0.64 |  0.62 |   10000 |  0 | 32 | 64 |
   | 0.999 |  0.87 |  0.88 |  0.89 |   10000 | 50 | 32 | 64 |
   | 0.978 |  1.09 |  1.08 |  1.08 |  100000 |  0 | 16 | 32 |
   | 0.987 |  1.29 |  1.32 |  1.34 |  100000 | 50 | 16 | 32 |
   | 0.989 |  1.10 |  1.09 |  1.10 |  100000 |  0 | 16 | 64 |
   | 0.994 |  1.48 |  1.49 |  1.46 |  100000 | 50 | 16 | 64 |
   | 0.977 |  0.98 |  0.99 |  0.98 |  100000 |  0 | 32 | 32 |
   | 0.987 |  1.33 |  1.35 |  1.34 |  100000 | 50 | 32 | 32 |
   | 0.989 |  1.13 |  1.14 |  1.13 |  100000 |  0 | 32 | 64 |
   | 0.994 |  1.55 |  1.55 |  1.53 |  100000 | 50 | 32 | 64 |
   | 0.957 |  1.48 |  1.52 |  1.49 | 1000000 |  0 | 16 | 32 |
   | 0.972 |  2.03 |  2.08 |  2.04 | 1000000 | 50 | 16 | 32 |
   | 0.976 |  1.70 |  1.73 |  1.71 | 1000000 |  0 | 16 | 64 |
   | 0.985 |  2.42 |  2.45 |  2.47 | 1000000 | 50 | 16 | 64 |
   | 0.959 |  1.67 |  1.65 |  1.66 | 1000000 |  0 | 32 | 32 |
   | 0.974 |  2.13 |  2.15 |  2.16 | 1000000 | 50 | 32 | 32 |
   | 0.978 |  1.89 |  1.84 |  1.89 | 1000000 |  0 | 32 | 64 |
   | 0.987 |  2.52 |  2.53 |  2.55 | 1000000 | 50 | 32 | 64 |
   
   ### enwiki (topK = 100, segment count = 1, executor = null)
   | recall | Sequential | SliceExecutor | ReduceOverhead | nDoc | fanout | maxConn | beamWidth |
   |-|-|-|-|-|-|-|-|
   | 0.941 |  0.22 |  0.21 |  0.24 |   10000 |  0 | 16 | 32 |
   | 0.970 |  0.24 |  0.24 |  0.25 |   10000 | 50 | 16 | 32 |
   | 0.965 |  0.20 |  0.19 |  0.20 |   10000 |  0 | 16 | 64 |
   | 0.984 |  0.28 |  0.27 |  0.28 |   10000 | 50 | 16 | 64 |
   | 0.941 |  0.18 |  0.17 |  0.18 |   10000 |  0 | 32 | 32 |
   | 0.970 |  0.24 |  0.23 |  0.23 |   10000 | 50 | 32 | 32 |
   | 0.966 |  0.20 |  0.20 |  0.20 |   10000 |  0 | 32 | 64 |
   | 0.985 |  0.28 |  0.27 |  0.26 |   10000 | 50 | 32 | 64 |
   | 0.909 |  0.27 |  0.27 |  0.27 |  100000 |  0 | 16 | 32 |
   | 0.945 |  0.38 |  0.36 |  0.37 |  100000 | 50 | 16 | 32 |
   | 0.944 |  0.32 |  0.30 |  0.30 |  100000 |  0 | 16 | 64 |
   | 0.969 |  0.43 |  0.41 |  0.42 |  100000 | 50 | 16 | 64 |
   | 0.914 |  0.28 |  0.28 |  0.29 |  100000 |  0 | 32 | 32 |
   | 0.948 |  0.39 |  0.38 |  0.38 |  100000 | 50 | 32 | 32 |
   | 0.949 |  0.30 |  0.30 |  0.32 |  100000 |  0 | 32 | 64 |
   | 0.972 |  0.44 |  0.41 |  0.40 |  100000 | 50 | 32 | 64 |
   | 0.870 |  0.35 |  0.34 |  0.35 | 1000000 |  0 | 16 | 32 |
   | 0.911 |  0.49 |  0.48 |  0.47 | 1000000 | 50 | 16 | 32 |
   | 0.913 |  0.40 |  0.40 |  0.41 | 1000000 |  0 | 16 | 64 |
   | 0.945 |  0.55 |  0.55 |  0.56 | 1000000 | 50 | 16 | 64 |
   | 0.881 |  0.38 |  0.39 |  0.38 | 1000000 |  0 | 32 | 32 |
   | 0.919 |  0.52 |  0.52 |  0.52 | 1000000 | 50 | 32 | 32 |
   | 0.923 |  0.45 |  0.45 |  0.46 | 1000000 |  0 | 32 | 64 |
   | 0.954 |  0.62 |  0.62 |  0.61 | 1000000 | 50 | 32 | 64 |
   
   There are a few places where it gives some speedup, but this seems to be too low (Note that there is also some logic duplication [here](https://github.com/kaivalnp/lucene/blob/concurrent-knn-reduce-overhead/lucene/core/src/java/org/apache/lucene/search/AbstractKnnVectorQuery.java#L88-L93) and [here](https://github.com/kaivalnp/lucene/blob/concurrent-knn-reduce-overhead/lucene/core/src/java/org/apache/lucene/search/AbstractKnnVectorQuery.java#L103-L108), which we would want to avoid, maybe by wrapping it in a callable. I tried that out locally and it was performing similar to worse)
   
   In the absence of an executor, we are setting it to `Runnable::run`, which performs the same tasks sequentially. My guess would be that its overhead is much lower compared to the search tasks, and IMO the readability earlier outweighs the separate `if` block
   
   Please let me know what you feel / if you had something else in mind?
   
   Edit: Sorry, links in this comment are now broken because they pointed to specific lines at the time of writing. Now that the underlying branch is updated, links point to unrelated 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


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

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


##########
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);
+
+    TopDocs[] perLeafResults =
+        tasks.stream()
+            .map(
+                task -> {
+                  try {
+                    return task.get();
+                  } catch (ExecutionException e) {
+                    throw new RuntimeException(e.getCause());
+                  } catch (InterruptedException e) {
+                    throw new RuntimeException(e);

Review Comment:
   Can you throw `ThreadInterruptedException` instead?



##########
lucene/core/src/test/org/apache/lucene/search/BaseKnnVectorQueryTestCase.java:
##########
@@ -210,7 +210,10 @@ public void testDimensionMismatch() throws IOException {
       IndexSearcher searcher = newSearcher(reader);
       AbstractKnnVectorQuery kvq = getKnnVectorQuery("field", new float[] {0}, 10);
       IllegalArgumentException e =
-          expectThrows(IllegalArgumentException.class, () -> searcher.search(kvq, 10));
+          expectThrows(
+              RuntimeException.class,
+              IllegalArgumentException.class,

Review Comment:
   Since IAE extends RuntimeExeption, it should be good to just do `expectThrows(RuntimeException.class, runnable)`?



##########
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);
+
+    TopDocs[] perLeafResults =
+        tasks.stream()
+            .map(
+                task -> {
+                  try {
+                    return task.get();
+                  } catch (ExecutionException e) {
+                    throw new RuntimeException(e.getCause());

Review Comment:
   I'm not confident it's safe to swallow the root exception and only report the cause? Would it work to `throw new RuntimeException(e)`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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


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


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

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


##########
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);
+
+    TopDocs[] perLeafResults =
+        tasks.stream()
+            .map(
+                task -> {
+                  try {
+                    return task.get();
+                  } catch (ExecutionException e) {
+                    throw new RuntimeException(e.getCause());

Review Comment:
   I think any exception thrown during the thread's execution is [(always?) wrapped](https://docs.oracle.com/en/java/javase/17/docs/api/java.base/java/util/concurrent/ExecutionException.html) in an `ExecutionException`
   
   I mainly used `getCause` for two reasons:
   - We would always have to throw a `RuntimeException(ExecutionException(actual Throwable))`, and the `ExecutionException` might be redundant there
   - `LuceneTestCase` (currently) allows checking [at most two wrapped exceptions](https://github.com/apache/lucene/blob/main/lucene/test-framework/src/java/org/apache/lucene/tests/util/LuceneTestCase.java#L2893-L2894), and the one above would have three
   
   However, I don't have any strong opinions on this and can write a function to check for three nested exceptions as well. Please let me know what you feel, and I'll update it accordingly



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] kaivalnp commented on pull request #12160: Concurrent rewrite for KnnVectorQuery

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

   Thanks! I wasn't sure where to add the entry..
   
   I have currently put it under Lucene 10 -> Optimizations
   Please let me know if you feel this should be someplace else


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