sohami commented on code in PR #12348:
URL: https://github.com/apache/lucene/pull/12348#discussion_r1223577406


##########
lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java:
##########
@@ -225,12 +217,24 @@ public IndexSearcher(IndexReaderContext context, Executor 
executor) {
     this(context, executor, getSliceExecutionControlPlane(executor));
   }
 
-  // Package private for testing
-  IndexSearcher(IndexReaderContext context, Executor executor, SliceExecutor 
sliceExecutor) {
+  /**
+   * Creates a searcher searching the provided top-level {@link 
IndexReaderContext}.
+   *
+   * <p>Given a non-<code>null</code> {@link Executor} and {@link 
SliceExecutor} this method runs
+   * searches for each segment separately, using the provided Executor. The 
provided {@link
+   * Executor} instance should be same as the one used in {@link 
SliceExecutor} NOTE: if you are
+   * using {@link NIOFSDirectory}, do not use the shutdownNow method of 
ExecutorService as this uses
+   * Thread.interrupt under-the-hood which can silently close file descriptors 
(see <a href=
+   * "https://issues.apache.org/jira/browse/LUCENE-2239";>LUCENE-2239</a>).
+   *
+   * @lucene.experimental
+   */
+  public IndexSearcher(IndexReaderContext context, Executor executor, 
SliceExecutor sliceExecutor) {

Review Comment:
   > I wonder if we should make this constructor public, or instead make the 
`getSliceExecutionControlPlane` method public. Would this second option work 
for what you need or would you still run into issues as it's called from the 
`IndexSearcher` constructor a bit like `slices`?
   
   Yes that would result in same issue
   
   > 
   > Otherwise I'd think we should look at having a single way, either provide 
an `Executor` or a `SliceExecutor`.
   
   There are already 
[constructors](https://github.com/apache/lucene/pull/12348/files#diff-c939535b77060b3d8ddba204a782225abbb49f51466a043e4cc944bb7a1c1320R216)
 which takes in `Executor`. So was unable to modify this one to just take in 
`SliceExecutor`. I also added an assertion for this constructor that `executor` 
instance is same as one in `SliceExecutor`



##########
lucene/core/src/java/org/apache/lucene/search/SliceExecutor.java:
##########
@@ -18,22 +18,41 @@
 package org.apache.lucene.search;
 
 import java.util.Collection;
+import java.util.List;
 import java.util.Objects;
 import java.util.concurrent.Executor;
 import java.util.concurrent.RejectedExecutionException;
+import org.apache.lucene.index.LeafReaderContext;
 
 /**
  * Executor which is responsible for execution of slices based on the current 
status of the system
  * and current system load
  */
-class SliceExecutor {
+public class SliceExecutor {
+
+  /** Thresholds for index slice allocation logic */
+  private static final int MAX_DOCS_PER_SLICE = 250_000;
+
+  private static final int MAX_SEGMENTS_PER_SLICE = 5;
+
   private final Executor executor;
 
-  SliceExecutor(Executor executor) {
+  public SliceExecutor(Executor executor) {
     this.executor = Objects.requireNonNull(executor, "Executor is null");
   }
 
-  final void invokeAll(Collection<? extends Runnable> tasks) {
+  /**
+   * method to segregate LeafReaderContexts amongst multiple slices using the 
default
+   * MAX_SEGMENTS_PER_SLICE and MAX_DOCUMENTS_PER_SLICE
+   *
+   * @param leaves LeafReaderContexts for this index
+   * @return computed slices
+   */
+  public LeafSlice[] computeSlices(List<LeafReaderContext> leaves) {
+    return IndexSearcher.slices(leaves, MAX_DOCS_PER_SLICE, 
MAX_SEGMENTS_PER_SLICE);

Review Comment:
   Didn't moved it to avoid breaking consumers as static `slices` method is 
`public`



##########
lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java:
##########
@@ -319,11 +323,10 @@ public QueryCachingPolicy getQueryCachingPolicy() {
 
   /**
    * Expert: Creates an array of leaf slices each holding a subset of the 
given leaves. Each {@link
-   * LeafSlice} is executed in a single thread. By default, segments with more 
than
-   * MAX_DOCS_PER_SLICE will get their own thread
+   * LeafSlice} is executed in a single thread.
    */
   protected LeafSlice[] slices(List<LeafReaderContext> leaves) {

Review Comment:
   I kept it like this to avoid breaking extensions already overriding this 
method. I wanted to backport this change to 9.x branch as well.



##########
lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java:
##########
@@ -882,27 +885,6 @@ public IndexReaderContext getTopReaderContext() {
     return readerContext;
   }
 
-  /**
-   * A class holding a subset of the {@link IndexSearcher}s leaf contexts to 
be executed within a
-   * single thread.
-   *
-   * @lucene.experimental
-   */
-  public static class LeafSlice {
-
-    /**
-     * The leaves that make up this slice.
-     *
-     * @lucene.experimental
-     */
-    public final LeafReaderContext[] leaves;
-
-    public LeafSlice(List<LeafReaderContext> leavesList) {
-      Collections.sort(leavesList, Comparator.comparingInt(l -> l.docBase));
-      this.leaves = leavesList.toArray(new LeafReaderContext[0]);
-    }
-  }

Review Comment:
   Ack I can move it back since it was getting used by `IndexSearcher` and 
`SliceExecutor`, I thought of moving it out.



##########
lucene/core/src/java/org/apache/lucene/search/SliceExecutor.java:
##########
@@ -55,4 +74,8 @@ boolean shouldExecuteOnCallerThread(int index, int numTasks) {
     // Execute last task on caller thread

Review Comment:
   Didn't see a need for that as it is internal to the implementation of 
`SliceExecutor`. A custom `SliceExecutor` can define its own implementation of 
`invokeAll` as needed.



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

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

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


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

Reply via email to