You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2020/02/27 04:56:33 UTC

[GitHub] [lucene-solr] atris opened a new pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

atris opened a new pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294
 
 
   This commit introduces a mechanism to control allocation of threads to slices planned for a query.
   The default implementation uses the size of backlog queue of the executor to determine if a slice
   should be allocated a new thread
   
   Supersedes #1214 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] atris commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r401565776
 
 

 ##########
 File path: lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java
 ##########
 @@ -70,13 +75,31 @@ public void setUp() throws Exception {
     }
     reader = iw.getReader();
     iw.close();
+
+    Random random = random();
+    RandomIndexWriter iw2 = new RandomIndexWriter(random(), dir2, newIndexWriterConfig().setMergePolicy(newLogMergePolicy()));
+    for (int i = 0; i < 100; i++) {
+      Document doc = new Document();
+      doc.add(newStringField("field", Integer.toString(i), Field.Store.NO));
+      doc.add(newStringField("field2", Boolean.toString(i % 2 == 0), Field.Store.NO));
+      doc.add(new SortedDocValuesField("field2", new BytesRef(Boolean.toString(i % 2 == 0))));
+      iw2.addDocument(doc);
+
+      if (random.nextBoolean()) {
+        iw2.commit();
+      }
+    }
+    reader2 = iw2.getReader();
+    iw2.close();
 
 Review comment:
   I did not want to touch the existing reader since it is used by many tests -- dont see any risk though, given the specifics of the tests that use the existing reader, should be safe. Updated, thanks

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] atris commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r384914799
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
 ##########
 @@ -211,6 +215,18 @@ public IndexSearcher(IndexReaderContext context, Executor executor) {
     assert context.isTopLevel: "IndexSearcher's ReaderContext must be topLevel for reader" + context.reader();
     reader = context.reader();
     this.executor = executor;
+    this.sliceExecutionControlPlane = executor == null ? null : getSliceExecutionControlPlane(executor);
+    this.readerContext = context;
+    leafContexts = context.leaves();
+    this.leafSlices = executor == null ? null : slices(leafContexts);
+  }
+
+  // Package private for testing
+  IndexSearcher(IndexReaderContext context, Executor executor, SliceExecutionControlPlane sliceExecutionControlPlane) {
 
 Review comment:
   To quote @jpountz :
   "this doesn't need an executor, does it?"
   
   It is needed since executor is a final member of IndexSearcher hence needs to be initialized? We could potentially always set it null since it anyways wont be used but that seemed a bit counter intuitive. WDYT?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] atris commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r391192155
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
 ##########
 @@ -211,6 +213,18 @@ public IndexSearcher(IndexReaderContext context, Executor executor) {
     assert context.isTopLevel: "IndexSearcher's ReaderContext must be topLevel for reader" + context.reader();
     reader = context.reader();
     this.executor = executor;
+    this.sliceExecutionControlPlane = executor == null ? null : getSliceExecutionControlPlane(executor);
+    this.readerContext = context;
+    leafContexts = context.leaves();
+    this.leafSlices = executor == null ? null : slices(leafContexts);
+  }
+
+  // Package private for testing
+  IndexSearcher(IndexReaderContext context, Executor executor, SliceExecutionControlPlane sliceExecutionControlPlane) {
+    assert context.isTopLevel: "IndexSearcher's ReaderContext must be topLevel for reader" + context.reader();
+    reader = context.reader();
+    this.executor = executor;
+    this.sliceExecutionControlPlane = executor == null ? null : sliceExecutionControlPlane;
 
 Review comment:
   +1, did not add it since it was a test method but I see your point. Added now, thanks!

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r390523963
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/SliceExecutor.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Future;
+import java.util.concurrent.FutureTask;
+import java.util.concurrent.RejectedExecutionException;
+
+/**
+ * Executor which is responsible
+ * for execution of slices based on the current status
+ * of the system and current system load
+ */
+class SliceExecutor {
+  private final Executor executor;
+
+  public SliceExecutor(Executor executor) {
+    this.executor = executor;
+  }
+
+  public <C> List<Future<C>> invokeAll(Collection<FutureTask<C>> tasks) {
+
+    if (tasks == null) {
+      throw new IllegalArgumentException("Tasks is null");
+    }
+
+    if (executor == null) {
+      throw new IllegalArgumentException("Executor is null");
+    }
+
+    List<Future<C>> futures = new ArrayList();
+
+    int i = 0;
+
+    for (FutureTask task : tasks) {
 
 Review comment:
   we should never use generic types without type parameters, can you address all these compilation warnings?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] atris commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r391181418
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/SliceExecutor.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Future;
+import java.util.concurrent.FutureTask;
+import java.util.concurrent.RejectedExecutionException;
+
+/**
+ * Executor which is responsible
+ * for execution of slices based on the current status
+ * of the system and current system load
+ */
+class SliceExecutor {
+  private final Executor executor;
+
+  public SliceExecutor(Executor executor) {
+    this.executor = executor;
+  }
+
+  public <C> List<Future<C>> invokeAll(Collection<FutureTask<C>> tasks) {
 
 Review comment:
   Agreed, fixed, thanks

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] atris commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r388075327
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/SliceExecutionControlPlane.java
 ##########
 @@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search;
+
+import java.util.Collection;
+
+/**
+ * Execution control plane which is responsible
+ * for execution of slices based on the current status
+ * of the system and current system load
+ */
+public interface SliceExecutionControlPlane<C, T extends Runnable> {
+  /**
+   * Invoke all slices that are allocated for the query
+   */
+  C invokeAll(Collection<T> tasks);
 
 Review comment:
   > Also- I'm curious if you saw any performance impact from the back pressure here?
   
   When testing with artificial CPU stress and a small threadpool size, the worst case performance tends towards sequential since all tasks will be scheduled on the caller thread

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r401555369
 
 

 ##########
 File path: lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java
 ##########
 @@ -70,13 +75,31 @@ public void setUp() throws Exception {
     }
     reader = iw.getReader();
     iw.close();
+
+    Random random = random();
+    RandomIndexWriter iw2 = new RandomIndexWriter(random(), dir2, newIndexWriterConfig().setMergePolicy(newLogMergePolicy()));
+    for (int i = 0; i < 100; i++) {
+      Document doc = new Document();
+      doc.add(newStringField("field", Integer.toString(i), Field.Store.NO));
+      doc.add(newStringField("field2", Boolean.toString(i % 2 == 0), Field.Store.NO));
+      doc.add(new SortedDocValuesField("field2", new BytesRef(Boolean.toString(i % 2 == 0))));
+      iw2.addDocument(doc);
+
+      if (random.nextBoolean()) {
+        iw2.commit();
+      }
+    }
+    reader2 = iw2.getReader();
+    iw2.close();
 
 Review comment:
   Would it be a problem to add these random commits to the existing reader?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] atris commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r385819447
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/SliceExecutionControlPlane.java
 ##########
 @@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search;
+
+import java.util.Collection;
+
+/**
+ * Execution control plane which is responsible
+ * for execution of slices based on the current status
+ * of the system and current system load
+ */
+public interface SliceExecutionControlPlane<C, T extends Runnable> {
+  /**
+   * Invoke all slices that are allocated for the query
+   */
+  C invokeAll(Collection<T> tasks);
 
 Review comment:
   Thanks for taking a look. I have fixed your comments. I have introduced the interface to allow easy extensibility. I agree that the current use case is with Future and FutureTask but that should not restrict the future use cases that are possible on top of the interface.
   
   Regarding the name, SliceRunner does not seem to convey the right message since the class can control the way the execution of the query happens without any internal directions hence it is vital to highlight that the execution of a query can be influenced by the decisions of this class.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r400490515
 
 

 ##########
 File path: lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java
 ##########
 @@ -347,4 +370,82 @@ public void execute(final Runnable runnable) {
       throw new RejectedExecutionException();
     }
   }
+
+  public void testQueueSizeBasedCP() throws Exception {
+    ThreadPoolExecutor service = new ThreadPoolExecutor(4, 4, 0L, TimeUnit.MILLISECONDS,
+        new LinkedBlockingQueue<Runnable>(),
+        new NamedThreadFactory("TestIndexSearcher"));
+
+    runCPTest(service, false);
+
+    TestUtil.shutdownExecutorService(service);
+  }
+
+  public void testRandomBlockingCP() throws Exception {
+    ThreadPoolExecutor service = new ThreadPoolExecutor(4, 4, 0L, TimeUnit.MILLISECONDS,
+        new LinkedBlockingQueue<Runnable>(),
+        new NamedThreadFactory("TestIndexSearcher"));
+
+    runCPTest(service, true);
+
+    TestUtil.shutdownExecutorService(service);
+  }
+
+  private void runCPTest(ThreadPoolExecutor service, boolean useRandomCP) throws Exception {
+    SliceExecutor sliceExecutor = useRandomCP == true ? new RandomBlockingSliceExecutor(service) :
+                                                              new QueueSizeBasedExecutor(service);
+
+    IndexSearcher searcher = new IndexSearcher(reader2.getContext(), service, sliceExecutor);
+
+    Query queries[] = new Query[] {
+        new MatchAllDocsQuery(),
+        new TermQuery(new Term("field", "1"))
+    };
+    Sort sorts[] = new Sort[] {
+        null,
+        new Sort(new SortField("field2", SortField.Type.STRING))
+    };
+    ScoreDoc afters[] = new ScoreDoc[] {
+        null,
+        new FieldDoc(0, 0f, new Object[] { new BytesRef("boo!") })
+    };
+
+    for (ScoreDoc after : afters) {
+      for (Query query : queries) {
+        for (Sort sort : sorts) {
+          searcher.search(query, Integer.MAX_VALUE);
+          searcher.searchAfter(after, query, Integer.MAX_VALUE);
+          if (sort != null) {
+            searcher.search(query, Integer.MAX_VALUE, sort);
+            searcher.search(query, Integer.MAX_VALUE, sort, true);
+            searcher.search(query, Integer.MAX_VALUE, sort, false);
+            searcher.searchAfter(after, query, Integer.MAX_VALUE, sort);
+            searcher.searchAfter(after, query, Integer.MAX_VALUE, sort, true);
+            searcher.searchAfter(after, query, Integer.MAX_VALUE, sort, false);
 
 Review comment:
   should we assert on the results?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] atris commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r384915037
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/SliceExecutionControlPlane.java
 ##########
 @@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search;
+
+import java.util.Collection;
+
+/**
+ * Execution control plane which is responsible
+ * for execution of slices based on the current status
+ * of the system and current system load
+ */
+public interface SliceExecutionControlPlane<C, T extends Runnable> {
+  /**
+   * Invoke all slices that are allocated for the query
+   */
+  C invokeAll(Collection<T> tasks);
 
 Review comment:
   To quote @jpountz 
   "the generics on this interface look over-engineered?"
   
   I am not sure as to what the correct generics should be. Could you please advice?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] atris commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r388075165
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/SliceExecutionControlPlane.java
 ##########
 @@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search;
+
+import java.util.Collection;
+
+/**
+ * Execution control plane which is responsible
+ * for execution of slices based on the current status
+ * of the system and current system load
+ */
+public interface SliceExecutionControlPlane<C, T extends Runnable> {
+  /**
+   * Invoke all slices that are allocated for the query
+   */
+  C invokeAll(Collection<T> tasks);
 
 Review comment:
   I do not have a very strong preference to the extensibility model of the interface -- so removed it per your recommendation. Let me know if you have any further comments or thoughts

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r388493819
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
 ##########
 @@ -211,6 +213,18 @@ public IndexSearcher(IndexReaderContext context, Executor executor) {
     assert context.isTopLevel: "IndexSearcher's ReaderContext must be topLevel for reader" + context.reader();
     reader = context.reader();
     this.executor = executor;
+    this.sliceExecutionControlPlane = executor == null ? null : getSliceExecutionControlPlane(executor);
+    this.readerContext = context;
+    leafContexts = context.leaves();
+    this.leafSlices = executor == null ? null : slices(leafContexts);
+  }
+
+  // Package private for testing
+  IndexSearcher(IndexReaderContext context, Executor executor, SliceExecutionControlPlane sliceExecutionControlPlane) {
+    assert context.isTopLevel: "IndexSearcher's ReaderContext must be topLevel for reader" + context.reader();
+    reader = context.reader();
+    this.executor = executor;
+    this.sliceExecutionControlPlane = executor == null ? null : sliceExecutionControlPlane;
 
 Review comment:
   it feels wrong to not take the one from the constructor?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r388497042
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
 ##########
 @@ -211,6 +215,18 @@ public IndexSearcher(IndexReaderContext context, Executor executor) {
     assert context.isTopLevel: "IndexSearcher's ReaderContext must be topLevel for reader" + context.reader();
     reader = context.reader();
     this.executor = executor;
+    this.sliceExecutionControlPlane = executor == null ? null : getSliceExecutionControlPlane(executor);
+    this.readerContext = context;
+    leafContexts = context.leaves();
+    this.leafSlices = executor == null ? null : slices(leafContexts);
+  }
+
+  // Package private for testing
+  IndexSearcher(IndexReaderContext context, Executor executor, SliceExecutionControlPlane sliceExecutionControlPlane) {
 
 Review comment:
   Is there anything we need to do with the executor that we couldn't do with the sliceExecutionControlPlane?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] atris commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r389038791
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
 ##########
 @@ -211,6 +213,18 @@ public IndexSearcher(IndexReaderContext context, Executor executor) {
     assert context.isTopLevel: "IndexSearcher's ReaderContext must be topLevel for reader" + context.reader();
     reader = context.reader();
     this.executor = executor;
+    this.sliceExecutionControlPlane = executor == null ? null : getSliceExecutionControlPlane(executor);
+    this.readerContext = context;
+    leafContexts = context.leaves();
+    this.leafSlices = executor == null ? null : slices(leafContexts);
+  }
+
+  // Package private for testing
+  IndexSearcher(IndexReaderContext context, Executor executor, SliceExecutionControlPlane sliceExecutionControlPlane) {
+    assert context.isTopLevel: "IndexSearcher's ReaderContext must be topLevel for reader" + context.reader();
+    reader = context.reader();
+    this.executor = executor;
+    this.sliceExecutionControlPlane = executor == null ? null : sliceExecutionControlPlane;
 
 Review comment:
   Not sure if I understood your point. The passed in instance is the one being assigned to the member?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r400480289
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
 ##########
 @@ -208,9 +210,22 @@ public IndexSearcher(IndexReader r, Executor executor) {
    * @lucene.experimental
    */
   public IndexSearcher(IndexReaderContext context, Executor executor) {
+    this(context, executor, getSliceExecutionControlPlane(executor));
+  }
+
+  // Package private for testing
+  IndexSearcher(IndexReaderContext context, Executor executor, SliceExecutor sliceExecutor) {
     assert context.isTopLevel: "IndexSearcher's ReaderContext must be topLevel for reader" + context.reader();
+    if (executor == null) {
+      assert sliceExecutor == null;
+    }
+    if (sliceExecutor == null) {
+      assert executor == null;
+    }
 
 Review comment:
   maybe do `assert (sliceExecutor == null) == (executor==null);` which is shorter and not harder to read?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] msokolov commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r385082544
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/DefaultSliceExecutionControlPlane.java
 ##########
 @@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Future;
+import java.util.concurrent.FutureTask;
+import java.util.concurrent.RejectedExecutionException;
+
+/**
+ * Implementation of SliceExecutionControlPlane with queue backpressure based thread allocation
+ */
+public class DefaultSliceExecutionControlPlane implements SliceExecutionControlPlane<List<Future>, FutureTask> {
+  private final Executor executor;
+
+  public DefaultSliceExecutionControlPlane(Executor executor) {
+    this.executor = executor;
+  }
+
+  @Override
+  public List<Future> invokeAll(Collection<FutureTask> tasks) {
+
+    if (tasks == null) {
+      throw new IllegalArgumentException("Tasks is null");
+    }
+
+    if (executor == null) {
+      throw new IllegalArgumentException("Executor is null");
+    }
+
+    List<Future> futures = new ArrayList();
+
+    int i = 0;
+
+    for (FutureTask task : tasks) {
+      boolean shouldExecuteOnCallerThread = false;
+
+      // Execute last task on caller thread
+      if (i == tasks.size() - 1) {
+        shouldExecuteOnCallerThread = true;
+      }
+
+      processTask(task, futures, shouldExecuteOnCallerThread);
+      ++i;
+    }
+
+    return futures;
+  }
+
+  // Helper method to execute a single task
+  protected void processTask(FutureTask task, List<Future> futures,
+                             boolean shouldExecuteOnCallerThread) {
 
 Review comment:
   The logic around this boolean is hard to read. I think it would be clearer if we restructure the logic a bit and avoid modifying an incoming boolean parameter

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r390514257
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
 ##########
 @@ -211,6 +213,18 @@ public IndexSearcher(IndexReaderContext context, Executor executor) {
     assert context.isTopLevel: "IndexSearcher's ReaderContext must be topLevel for reader" + context.reader();
     reader = context.reader();
     this.executor = executor;
+    this.sliceExecutionControlPlane = executor == null ? null : getSliceExecutionControlPlane(executor);
+    this.readerContext = context;
+    leafContexts = context.leaves();
+    this.leafSlices = executor == null ? null : slices(leafContexts);
+  }
+
+  // Package private for testing
+  IndexSearcher(IndexReaderContext context, Executor executor, SliceExecutionControlPlane sliceExecutionControlPlane) {
+    assert context.isTopLevel: "IndexSearcher's ReaderContext must be topLevel for reader" + context.reader();
+    reader = context.reader();
+    this.executor = executor;
+    this.sliceExecutionControlPlane = executor == null ? null : sliceExecutionControlPlane;
 
 Review comment:
   My point was that it sounds like a bug on the caller of this constructor to pass a null executor and a non-null sliceExecutionControlPlane? So I'd rather have validation around it rather than be lenient and ignore the provided sliceExecutionControlPlane if the executor is null?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] msokolov commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r385083507
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/DefaultSliceExecutionControlPlane.java
 ##########
 @@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Future;
+import java.util.concurrent.FutureTask;
+import java.util.concurrent.RejectedExecutionException;
+
+/**
+ * Implementation of SliceExecutionControlPlane with queue backpressure based thread allocation
+ */
+public class DefaultSliceExecutionControlPlane implements SliceExecutionControlPlane<List<Future>, FutureTask> {
+  private final Executor executor;
+
+  public DefaultSliceExecutionControlPlane(Executor executor) {
+    this.executor = executor;
+  }
+
+  @Override
+  public List<Future> invokeAll(Collection<FutureTask> tasks) {
+
+    if (tasks == null) {
+      throw new IllegalArgumentException("Tasks is null");
+    }
+
+    if (executor == null) {
+      throw new IllegalArgumentException("Executor is null");
+    }
+
+    List<Future> futures = new ArrayList();
+
+    int i = 0;
+
+    for (FutureTask task : tasks) {
+      boolean shouldExecuteOnCallerThread = false;
+
+      // Execute last task on caller thread
+      if (i == tasks.size() - 1) {
+        shouldExecuteOnCallerThread = true;
+      }
+
+      processTask(task, futures, shouldExecuteOnCallerThread);
+      ++i;
+    }
+
+    return futures;
+  }
+
+  // Helper method to execute a single task
+  protected void processTask(FutureTask task, List<Future> futures,
+                             boolean shouldExecuteOnCallerThread) {
+    if (task == null) {
+      throw new IllegalArgumentException("Input is null");
+    }
+
+    if (!shouldExecuteOnCallerThread) {
+      try {
+        executor.execute(task);
+      } catch (RejectedExecutionException e) {
+        // Execute on caller thread
+        shouldExecuteOnCallerThread = true;
+      }
+    }
+
+    if (shouldExecuteOnCallerThread) {
+      try {
+        task.run();
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    if (!shouldExecuteOnCallerThread) {
 
 Review comment:
   could we add this to the first if block? ie executor.execute() .. futures.add() ...?  Then use early return and you don't need to modify the incoming parameter

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] atris commented on issue #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
atris commented on issue #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#issuecomment-597808136
 
 
   @jpountz Fixed, please see and let me know your thoughts.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] atris commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r401544477
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/SliceExecutor.java
 ##########
 @@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search;
+
+import java.util.Collection;
+import java.util.concurrent.Executor;
+import java.util.concurrent.RejectedExecutionException;
+
+/**
+ * Executor which is responsible
+ * for execution of slices based on the current status
+ * of the system and current system load
+ */
+class SliceExecutor {
+  private final Executor executor;
+
+  public SliceExecutor(Executor executor) {
+    this.executor = executor;
+  }
+
+  public void invokeAll(Collection<? extends Runnable> tasks) {
+
+    if (tasks == null) {
+      throw new IllegalArgumentException("Tasks is null");
+    }
+
+    if (executor == null) {
+      throw new IllegalArgumentException("Executor is null");
+    }
+
+    int i = 0;
+
+    for (Runnable task : tasks) {
+      boolean shouldExecuteOnCallerThread = false;
+
+      // Execute last task on caller thread
+      if (i == tasks.size() - 1) {
+        shouldExecuteOnCallerThread = true;
+      }
+
+      processTask(task, shouldExecuteOnCallerThread);
+      ++i;
+    };
+  }
+
+  // Helper method to execute a single task
+  protected void processTask(final Runnable task,
+                             final boolean shouldExecuteOnCallerThread) {
+    if (task == null) {
+      throw new IllegalArgumentException("Input is null");
+    }
+
+    if (!shouldExecuteOnCallerThread) {
+      try {
+        executor.execute(task);
+
+        return;
+      } catch (RejectedExecutionException e) {
+        // Execute on caller thread
+      }
+    }
+
+    runTaskOnCallerThread(task);
+  }
+
+  // Private helper method to run a task on the caller thread
+  private void runTaskOnCallerThread(Runnable task) {
+    task.run();
+  }
 
 Review comment:
   Fair point -- I dont know what I was thinking :)

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r390525903
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/SliceExecutor.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Future;
+import java.util.concurrent.FutureTask;
+import java.util.concurrent.RejectedExecutionException;
+
+/**
+ * Executor which is responsible
+ * for execution of slices based on the current status
+ * of the system and current system load
+ */
+class SliceExecutor {
+  private final Executor executor;
+
+  public SliceExecutor(Executor executor) {
+    this.executor = executor;
+  }
+
+  public <C> List<Future<C>> invokeAll(Collection<FutureTask<C>> tasks) {
+
+    if (tasks == null) {
+      throw new IllegalArgumentException("Tasks is null");
+    }
+
+    if (executor == null) {
+      throw new IllegalArgumentException("Executor is null");
+    }
+
+    List<Future<C>> futures = new ArrayList();
+
+    int i = 0;
+
+    for (FutureTask task : tasks) {
+      boolean shouldExecuteOnCallerThread = false;
+
+      // Execute last task on caller thread
+      if (i == tasks.size() - 1) {
+        shouldExecuteOnCallerThread = true;
+      }
+
+      processTask(task, futures, shouldExecuteOnCallerThread);
+      ++i;
+    }
+
+    return futures;
+  }
+
+  // Helper method to execute a single task
+  protected <C> void processTask(final FutureTask<C> task, final List<Future<C>> futures,
+                             final boolean shouldExecuteOnCallerThread) {
+    if (task == null) {
+      throw new IllegalArgumentException("Input is null");
+    }
+
+    if (!shouldExecuteOnCallerThread) {
+      try {
+        executor.execute(task);
+        futures.add(task);
+
+        return;
+      } catch (RejectedExecutionException e) {
+        // Execute on caller thread
+      }
+    }
+
+    runTaskOnCallerThread(task);
+
+    try {
+      futures.add(CompletableFuture.completedFuture(task.get()));
 
 Review comment:
   this has the same effect as `futures.add(task)` unless 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r388497511
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
 ##########
 @@ -662,34 +676,19 @@ public TopFieldDocs reduce(Collection<TopFieldCollector> collectors) throws IOEx
       }
       query = rewrite(query);
       final Weight weight = createWeight(query, scoreMode, 1);
-      final List<Future<C>> topDocsFutures = new ArrayList<>(leafSlices.length);
-      for (int i = 0; i < leafSlices.length - 1; ++i) {
+      final List<FutureTask> listTasks = new ArrayList<>();
 
 Review comment:
   Let's avoid introducing warnings about generics, FutureTask needs to be parameterized?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r388498188
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/SliceExecutionControlPlane.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Future;
+import java.util.concurrent.FutureTask;
+import java.util.concurrent.RejectedExecutionException;
+
+/**
+ * Execution control plane which is responsible
+ * for execution of slices based on the current status
+ * of the system and current system load
+ */
+class SliceExecutionControlPlane {
 
 Review comment:
   nit: I'd prefer a simpler name, e.g. `SliceExecutor`

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r390515024
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
 ##########
 @@ -211,6 +213,18 @@ public IndexSearcher(IndexReaderContext context, Executor executor) {
     assert context.isTopLevel: "IndexSearcher's ReaderContext must be topLevel for reader" + context.reader();
     reader = context.reader();
     this.executor = executor;
+    this.sliceExecutor = executor == null ? null : getSliceExecutionControlPlane(executor);
+    this.readerContext = context;
+    leafContexts = context.leaves();
+    this.leafSlices = executor == null ? null : slices(leafContexts);
 
 Review comment:
   maybe this should delegate to the below constructor?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] atris merged pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
atris merged pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294
 
 
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r400486985
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/SliceExecutor.java
 ##########
 @@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search;
+
+import java.util.Collection;
+import java.util.concurrent.Executor;
+import java.util.concurrent.RejectedExecutionException;
+
+/**
+ * Executor which is responsible
+ * for execution of slices based on the current status
+ * of the system and current system load
+ */
+class SliceExecutor {
+  private final Executor executor;
+
+  public SliceExecutor(Executor executor) {
+    this.executor = executor;
+  }
+
+  public void invokeAll(Collection<? extends Runnable> tasks) {
+
+    if (tasks == null) {
+      throw new IllegalArgumentException("Tasks is null");
+    }
+
+    if (executor == null) {
+      throw new IllegalArgumentException("Executor is null");
+    }
+
+    int i = 0;
+
+    for (Runnable task : tasks) {
+      boolean shouldExecuteOnCallerThread = false;
+
+      // Execute last task on caller thread
+      if (i == tasks.size() - 1) {
+        shouldExecuteOnCallerThread = true;
+      }
+
+      processTask(task, shouldExecuteOnCallerThread);
+      ++i;
+    };
+  }
+
+  // Helper method to execute a single task
+  protected void processTask(final Runnable task,
+                             final boolean shouldExecuteOnCallerThread) {
+    if (task == null) {
+      throw new IllegalArgumentException("Input is null");
+    }
+
+    if (!shouldExecuteOnCallerThread) {
+      try {
+        executor.execute(task);
+
+        return;
+      } catch (RejectedExecutionException e) {
+        // Execute on caller thread
+      }
+    }
+
+    runTaskOnCallerThread(task);
+  }
+
+  // Private helper method to run a task on the caller thread
+  private void runTaskOnCallerThread(Runnable task) {
+    task.run();
+  }
 
 Review comment:
   do we really need a helper for this?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] msokolov commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r386109193
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/SliceExecutionControlPlane.java
 ##########
 @@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search;
+
+import java.util.Collection;
+
+/**
+ * Execution control plane which is responsible
+ * for execution of slices based on the current status
+ * of the system and current system load
+ */
+public interface SliceExecutionControlPlane<C, T extends Runnable> {
+  /**
+   * Invoke all slices that are allocated for the query
+   */
+  C invokeAll(Collection<T> tasks);
 
 Review comment:
   This is an internal detail of IndexSearcher, right? We're always free to change the method signatures later (if we keep the classes package-private - we should!). Maybe it would help if you were to explain what extension you have in mind.
   
   By the way, using force push makes it more difficult for reviewers since we can't easily see what changed from one version to the next. In general it's better to push your commits and then squash-merge them at the end (github will even do this for you I think)

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r390522773
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
 ##########
 @@ -933,6 +932,13 @@ public Executor getExecutor() {
     return executor;
   }
 
+  /**
+   * Returns this searchers slice execution control plane or <code>null</code> if no executor was provided
+   */
+  public SliceExecutor getSliceExecutor() {
 
 Review comment:
   we shouldn't make this method public if it returns a pkg-private class, let's make the method pkg-private too? Or even remove it entirely as I'm not seeing any call site for it?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r401583032
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
 ##########
 @@ -208,9 +210,21 @@ public IndexSearcher(IndexReader r, Executor executor) {
    * @lucene.experimental
    */
   public IndexSearcher(IndexReaderContext context, Executor executor) {
+    this(context, executor, getSliceExecutionControlPlane(executor));
+  }
+
+  // Package private for testing
+  IndexSearcher(IndexReaderContext context, Executor executor, SliceExecutor sliceExecutor) {
     assert context.isTopLevel: "IndexSearcher's ReaderContext must be topLevel for reader" + context.reader();
+    if (executor == null) {
+      assert sliceExecutor == null;
+    }
 
 Review comment:
   this is covered by the below assert already?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r400490859
 
 

 ##########
 File path: lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java
 ##########
 @@ -347,4 +370,82 @@ public void execute(final Runnable runnable) {
       throw new RejectedExecutionException();
     }
   }
+
+  public void testQueueSizeBasedCP() throws Exception {
 
 Review comment:
   what is CP the acronym of?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] msokolov commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r386109250
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/SliceExecutionControlPlane.java
 ##########
 @@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search;
+
+import java.util.Collection;
+
+/**
+ * Execution control plane which is responsible
+ * for execution of slices based on the current status
+ * of the system and current system load
+ */
+public interface SliceExecutionControlPlane<C, T extends Runnable> {
+  /**
+   * Invoke all slices that are allocated for the query
+   */
+  C invokeAll(Collection<T> tasks);
 
 Review comment:
   Also- I'm curious if you saw any performance impact from the back pressure here?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] atris commented on issue #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
atris commented on issue #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#issuecomment-601995664
 
 
   @jpountz Let me know if this looks fine.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] msokolov commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r385080416
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/SliceExecutionControlPlane.java
 ##########
 @@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search;
+
+import java.util.Collection;
+
+/**
+ * Execution control plane which is responsible
+ * for execution of slices based on the current status
+ * of the system and current system load
+ */
+public interface SliceExecutionControlPlane<C, T extends Runnable> {
+  /**
+   * Invoke all slices that are allocated for the query
+   */
+  C invokeAll(Collection<T> tasks);
 
 Review comment:
   I think it does not need to be generic at all since the only use case is for Future and FutureTask. In fact the interface serves no visible purpose; we could simply rename DefaultSliceExecutionControlPlane to SliceExecutionControlPlane and nothing else would change. Since we're on naming, the naming seems rather grandiose for my tastes. I tend to think of a ControlPlane as a component in a distributed system, but I struggle to come up with anything better. Maybe SliceRunner?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r400483056
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/QueueSizeBasedExecutor.java
 ##########
 @@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search;
+
+import java.util.Collection;
+import java.util.concurrent.ThreadPoolExecutor;
+
+/**
+ * Derivative of SliceExecutor that controls the number of active threads
+ * that are used for a single query. At any point, no more than (maximum pool size of the executor * LIMITING_FACTOR)
+ * threads should be active. If the limit is exceeded, further segments are searched on the caller thread
 
 Review comment:
   I believe you meant tasks, not threads.
   ```suggestion
    * tasks should be active. If the limit is exceeded, further segments are searched on the caller thread
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r390523437
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/SliceExecutor.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Future;
+import java.util.concurrent.FutureTask;
+import java.util.concurrent.RejectedExecutionException;
+
+/**
+ * Executor which is responsible
+ * for execution of slices based on the current status
+ * of the system and current system load
+ */
+class SliceExecutor {
+  private final Executor executor;
+
+  public SliceExecutor(Executor executor) {
+    this.executor = executor;
+  }
+
+  public <C> List<Future<C>> invokeAll(Collection<FutureTask<C>> tasks) {
 
 Review comment:
   I wonder whether this is the right API. We could change the return type to `void` and use `Runnable` instead of `FutureTask` and that would still work, right? The return value isn't really useful since it has the same content as the input collection? So what about making it just: `public void invokeAll(Collection<? extends Runnable> tasks)`?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r400491077
 
 

 ##########
 File path: lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java
 ##########
 @@ -70,13 +75,31 @@ public void setUp() throws Exception {
     }
     reader = iw.getReader();
     iw.close();
+
+    Random random = random();
+    RandomIndexWriter iw2 = new RandomIndexWriter(random(), dir2, newIndexWriterConfig().setMergePolicy(newLogMergePolicy()));
+    for (int i = 0; i < 100; i++) {
+      Document doc = new Document();
+      doc.add(newStringField("field", Integer.toString(i), Field.Store.NO));
+      doc.add(newStringField("field2", Boolean.toString(i % 2 == 0), Field.Store.NO));
+      doc.add(new SortedDocValuesField("field2", new BytesRef(Boolean.toString(i % 2 == 0))));
+      iw2.addDocument(doc);
+
+      if (random.nextBoolean()) {
+        iw2.commit();
+      }
+    }
+    reader2 = iw2.getReader();
+    iw2.close();
 
 Review comment:
   why are we not reusing reader1?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] atris commented on issue #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
atris commented on issue #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#issuecomment-607222229
 
 
   @jpountz Updated. Please see and let me know your comments and thoughts

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] atris commented on issue #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
atris commented on issue #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#issuecomment-595884480
 
 
   @jpountz Raised another iteration, please let me know your thoughts and comments.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r401584443
 
 

 ##########
 File path: lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java
 ##########
 @@ -347,4 +354,89 @@ public void execute(final Runnable runnable) {
       throw new RejectedExecutionException();
     }
   }
+
+  public void testQueueSizeBasedSliceExecutor() throws Exception {
+    ThreadPoolExecutor service = new ThreadPoolExecutor(4, 4, 0L, TimeUnit.MILLISECONDS,
+        new LinkedBlockingQueue<Runnable>(),
+        new NamedThreadFactory("TestIndexSearcher"));
+
+    runSliceExecutorTest(service, false);
+
+    TestUtil.shutdownExecutorService(service);
+  }
+
+  public void testRandomBlockingSliceExecutor() throws Exception {
+    ThreadPoolExecutor service = new ThreadPoolExecutor(4, 4, 0L, TimeUnit.MILLISECONDS,
+        new LinkedBlockingQueue<Runnable>(),
+        new NamedThreadFactory("TestIndexSearcher"));
+
+    runSliceExecutorTest(service, true);
+
+    TestUtil.shutdownExecutorService(service);
+  }
+
+  private void runSliceExecutorTest(ThreadPoolExecutor service, boolean useRandomSliceExecutor) throws Exception {
+    SliceExecutor sliceExecutor = useRandomSliceExecutor == true ? new RandomBlockingSliceExecutor(service) :
+                                                              new QueueSizeBasedExecutor(service);
+
+    IndexSearcher searcher = new IndexSearcher(reader.getContext(), service, sliceExecutor);
+
+    Query queries[] = new Query[] {
+        new MatchAllDocsQuery(),
+        new TermQuery(new Term("field", "1"))
+    };
+    Sort sorts[] = new Sort[] {
+        null,
+        new Sort(new SortField("field2", SortField.Type.STRING))
+    };
+    ScoreDoc afters[] = new ScoreDoc[] {
+        null,
+        new FieldDoc(0, 0f, new Object[] { new BytesRef("boo!") })
+    };
+
+    for (ScoreDoc after : afters) {
+      for (Query query : queries) {
+        for (Sort sort : sorts) {
+          searcher.search(query, Integer.MAX_VALUE);
+          searcher.searchAfter(after, query, Integer.MAX_VALUE);
+          if (sort != null) {
+            TopDocs topDocs = searcher.search(query, Integer.MAX_VALUE, sort);
+            assert topDocs.totalHits.value > 0;
+
+            topDocs = searcher.search(query, Integer.MAX_VALUE, sort, true);
+            assert topDocs.totalHits.value > 0;
+
+            topDocs = searcher.search(query, Integer.MAX_VALUE, sort, false);
+            assert topDocs.totalHits.value > 0;
+
+            topDocs = searcher.searchAfter(after, query, Integer.MAX_VALUE, sort);
+            assert topDocs.totalHits.value > 0;
+
+            topDocs = searcher.searchAfter(after, query, Integer.MAX_VALUE, sort, true);
+            assert topDocs.totalHits.value > 0;
+
+            topDocs = searcher.searchAfter(after, query, Integer.MAX_VALUE, sort, false);
+            assert topDocs.totalHits.value > 0;
 
 Review comment:
   we should use assertTrue rather than assert in tests, since the latter only runs when assertions are enabled, and it can be useful to run tests with assertions disabled to make sure that the logic remains correct when assertions are off

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r390521799
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/SliceExecutor.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Future;
+import java.util.concurrent.FutureTask;
+import java.util.concurrent.RejectedExecutionException;
+
+/**
+ * Executor which is responsible
+ * for execution of slices based on the current status
+ * of the system and current system load
+ */
+class SliceExecutor {
+  private final Executor executor;
+
+  public SliceExecutor(Executor executor) {
+    this.executor = executor;
+  }
+
+  public <C> List<Future<C>> invokeAll(Collection<FutureTask<C>> tasks) {
+
+    if (tasks == null) {
+      throw new IllegalArgumentException("Tasks is null");
+    }
+
+    if (executor == null) {
+      throw new IllegalArgumentException("Executor is null");
+    }
+
+    List<Future<C>> futures = new ArrayList();
+
+    int i = 0;
+
+    for (FutureTask task : tasks) {
+      boolean shouldExecuteOnCallerThread = false;
+
+      // Execute last task on caller thread
+      if (i == tasks.size() - 1) {
+        shouldExecuteOnCallerThread = true;
+      }
+
+      processTask(task, futures, shouldExecuteOnCallerThread);
+      ++i;
+    }
+
+    return futures;
+  }
+
+  // Helper method to execute a single task
+  protected <C> void processTask(final FutureTask<C> task, final List<Future<C>> futures,
+                             final boolean shouldExecuteOnCallerThread) {
+    if (task == null) {
+      throw new IllegalArgumentException("Input is null");
+    }
+
+    if (!shouldExecuteOnCallerThread) {
+      try {
+        executor.execute(task);
+        futures.add(task);
+
+        return;
+      } catch (RejectedExecutionException e) {
+        // Execute on caller thread
+      }
+    }
+
+    runTaskOnCallerThread(task);
+
+    try {
+      futures.add(CompletableFuture.completedFuture(task.get()));
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  // Private helper method to run a task on the caller thread
+  private void runTaskOnCallerThread(FutureTask task) {
+    try {
+      task.run();
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
 
 Review comment:
   we don't need this catch block as task.run() doesn't declare any non-runtime exception?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] atris commented on issue #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
atris commented on issue #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#issuecomment-607309069
 
 
   Thank you @jpountz 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] atris commented on issue #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
atris commented on issue #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#issuecomment-597139106
 
 
   @jpountz Any thoughts on this one?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] msokolov commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r385082712
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/DefaultSliceExecutionControlPlane.java
 ##########
 @@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Future;
+import java.util.concurrent.FutureTask;
+import java.util.concurrent.RejectedExecutionException;
+
+/**
+ * Implementation of SliceExecutionControlPlane with queue backpressure based thread allocation
+ */
+public class DefaultSliceExecutionControlPlane implements SliceExecutionControlPlane<List<Future>, FutureTask> {
+  private final Executor executor;
+
+  public DefaultSliceExecutionControlPlane(Executor executor) {
+    this.executor = executor;
+  }
+
+  @Override
+  public List<Future> invokeAll(Collection<FutureTask> tasks) {
+
+    if (tasks == null) {
+      throw new IllegalArgumentException("Tasks is null");
+    }
+
+    if (executor == null) {
+      throw new IllegalArgumentException("Executor is null");
+    }
+
+    List<Future> futures = new ArrayList();
+
+    int i = 0;
+
+    for (FutureTask task : tasks) {
+      boolean shouldExecuteOnCallerThread = false;
+
+      // Execute last task on caller thread
+      if (i == tasks.size() - 1) {
+        shouldExecuteOnCallerThread = true;
+      }
+
+      processTask(task, futures, shouldExecuteOnCallerThread);
+      ++i;
+    }
+
+    return futures;
+  }
+
+  // Helper method to execute a single task
+  protected void processTask(FutureTask task, List<Future> futures,
+                             boolean shouldExecuteOnCallerThread) {
+    if (task == null) {
+      throw new IllegalArgumentException("Input is null");
+    }
+
+    if (!shouldExecuteOnCallerThread) {
+      try {
+        executor.execute(task);
+      } catch (RejectedExecutionException e) {
+        // Execute on caller thread
+        shouldExecuteOnCallerThread = true;
+      }
+    }
+
+    if (shouldExecuteOnCallerThread) {
+      try {
+        task.run();
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    if (!shouldExecuteOnCallerThread) {
 
 Review comment:
   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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] atris commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
atris commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r401542281
 
 

 ##########
 File path: lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java
 ##########
 @@ -70,13 +75,31 @@ public void setUp() throws Exception {
     }
     reader = iw.getReader();
     iw.close();
+
+    Random random = random();
+    RandomIndexWriter iw2 = new RandomIndexWriter(random(), dir2, newIndexWriterConfig().setMergePolicy(newLogMergePolicy()));
+    for (int i = 0; i < 100; i++) {
+      Document doc = new Document();
+      doc.add(newStringField("field", Integer.toString(i), Field.Store.NO));
+      doc.add(newStringField("field2", Boolean.toString(i % 2 == 0), Field.Store.NO));
+      doc.add(new SortedDocValuesField("field2", new BytesRef(Boolean.toString(i % 2 == 0))));
+      iw2.addDocument(doc);
+
+      if (random.nextBoolean()) {
+        iw2.commit();
+      }
+    }
+    reader2 = iw2.getReader();
+    iw2.close();
 
 Review comment:
   reader2 has random commits baked into it to ensure random slice distributions each time the test is run.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1294: LUCENE-9074: Slice Allocation Control Plane For Concurrent Searches
URL: https://github.com/apache/lucene-solr/pull/1294#discussion_r400489607
 
 

 ##########
 File path: lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java
 ##########
 @@ -347,4 +370,82 @@ public void execute(final Runnable runnable) {
       throw new RejectedExecutionException();
     }
   }
+
+  public void testQueueSizeBasedCP() throws Exception {
+    ThreadPoolExecutor service = new ThreadPoolExecutor(4, 4, 0L, TimeUnit.MILLISECONDS,
+        new LinkedBlockingQueue<Runnable>(),
+        new NamedThreadFactory("TestIndexSearcher"));
+
+    runCPTest(service, false);
+
+    TestUtil.shutdownExecutorService(service);
+  }
+
+  public void testRandomBlockingCP() throws Exception {
+    ThreadPoolExecutor service = new ThreadPoolExecutor(4, 4, 0L, TimeUnit.MILLISECONDS,
+        new LinkedBlockingQueue<Runnable>(),
+        new NamedThreadFactory("TestIndexSearcher"));
+
+    runCPTest(service, true);
+
+    TestUtil.shutdownExecutorService(service);
+  }
+
+  private void runCPTest(ThreadPoolExecutor service, boolean useRandomCP) throws Exception {
+    SliceExecutor sliceExecutor = useRandomCP == true ? new RandomBlockingSliceExecutor(service) :
+                                                              new QueueSizeBasedExecutor(service);
+
+    IndexSearcher searcher = new IndexSearcher(reader2.getContext(), service, sliceExecutor);
+
+    Query queries[] = new Query[] {
+        new MatchAllDocsQuery(),
+        new TermQuery(new Term("field", "1"))
+    };
+    Sort sorts[] = new Sort[] {
+        null,
+        new Sort(new SortField("field2", SortField.Type.STRING))
+    };
+    ScoreDoc afters[] = new ScoreDoc[] {
+        null,
+        new FieldDoc(0, 0f, new Object[] { new BytesRef("boo!") })
+    };
+
+    for (ScoreDoc after : afters) {
+      for (Query query : queries) {
+        for (Sort sort : sorts) {
+          searcher.search(query, Integer.MAX_VALUE);
+          searcher.searchAfter(after, query, Integer.MAX_VALUE);
+          if (sort != null) {
+            searcher.search(query, Integer.MAX_VALUE, sort);
+            searcher.search(query, Integer.MAX_VALUE, sort, true);
+            searcher.search(query, Integer.MAX_VALUE, sort, false);
+            searcher.searchAfter(after, query, Integer.MAX_VALUE, sort);
+            searcher.searchAfter(after, query, Integer.MAX_VALUE, sort, true);
+            searcher.searchAfter(after, query, Integer.MAX_VALUE, sort, false);
+          }
+        }
+      }
+    }
+  }
+
+  private class RandomBlockingSliceExecutor extends SliceExecutor {
+
+    public RandomBlockingSliceExecutor(Executor executor) {
+      super(executor);
+    }
+
+    @Override
+    public void invokeAll(Collection<? extends Runnable> tasks){
+
+      for (Runnable task : tasks) {
+        boolean shouldExecuteOnCallerThread = false;
+
+        if (random().nextBoolean()) {
+          shouldExecuteOnCallerThread = true;
+        }
 
 Review comment:
   or just `boolean shouldExecuteOnCallerThread = random().nextBoolean()`?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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