You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2019/06/18 12:56:58 UTC

[lucene-solr] branch master updated: LUCENE-8865: Use incoming thread for execution if IndexSearcher has an executor (#725)

This is an automated email from the ASF dual-hosted git repository.

simonw pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/master by this push:
     new 60f3b25  LUCENE-8865: Use incoming thread for execution if IndexSearcher has an executor (#725)
60f3b25 is described below

commit 60f3b25d0600b1a1d3ea93f6f31d3bff02d041a9
Author: Simon Willnauer <si...@apache.org>
AuthorDate: Tue Jun 18 14:56:51 2019 +0200

    LUCENE-8865: Use incoming thread for execution if IndexSearcher has an executor (#725)
    
    Today we don't utilize the incoming thread for a search when IndexSearcher
    has an executor. This thread is only idling but can be used to execute a search
    once all other collectors are dispatched.
---
 lucene/CHANGES.txt                                 |  4 ++++
 .../org/apache/lucene/search/IndexSearcher.java    | 22 +++++++++++-----------
 2 files changed, 15 insertions(+), 11 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 122358d..8712800 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -108,6 +108,10 @@ Optimizations
 * LUCENE-8796: Use exponential search instead of binary search in
   IntArrayDocIdSet#advance method (Luca Cavanna via Adrien Grand)
 
+* LUCENE-8865: Use incoming thread for execution if IndexSearcher has an executor.
+  Now caller threads execute at least one search on an index even if there is
+  an executor provided to minimize thread context switching. (Simon Willnauer)
+
 Test Framework
 
 * LUCENE-8825: CheckHits now display the shard index in case of mismatch
diff --git a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
index 1748dd4..9fb0263 100644
--- a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
+++ b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
@@ -26,7 +26,7 @@ import java.util.Comparator;
 import java.util.List;
 import java.util.Objects;
 import java.util.Set;
-import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
@@ -613,7 +613,7 @@ public class IndexSearcher {
   * @lucene.experimental
   */
   public <C extends Collector, T> T search(Query query, CollectorManager<C, T> collectorManager) throws IOException {
-    if (executor == null) {
+    if (executor == null || leafSlices.length <= 1) {
       final C collector = collectorManager.newCollector();
       search(query, collector);
       return collectorManager.reduce(Collections.singletonList(collector));
@@ -636,18 +636,19 @@ public class IndexSearcher {
       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; ++i) {
+      for (int i = 0; i < leafSlices.length - 1; ++i) {
         final LeafReaderContext[] leaves = leafSlices[i].leaves;
         final C collector = collectors.get(i);
-        topDocsFutures.add(executor.submit(new Callable<C>() {
-          @Override
-          public C call() throws Exception {
-            search(Arrays.asList(leaves), weight, collector);
-            return collector;
-          }
+        topDocsFutures.add(executor.submit(() -> {
+          search(Arrays.asList(leaves), weight, collector);
+          return collector;
         }));
       }
-
+      final LeafReaderContext[] leaves = leafSlices[leafSlices.length - 1].leaves;
+      final C collector = collectors.get(leafSlices.length - 1);
+      // execute the last on the caller thread
+      search(Arrays.asList(leaves), weight, collector);
+      topDocsFutures.add(CompletableFuture.completedFuture(collector));
       final List<C> collectedCollectors = new ArrayList<>();
       for (Future<C> future : topDocsFutures) {
         try {
@@ -658,7 +659,6 @@ public class IndexSearcher {
           throw new RuntimeException(e);
         }
       }
-
       return collectorManager.reduce(collectors);
     }
   }