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-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org