mirror of https://github.com/apache/lucene.git
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.
This commit is contained in:
parent
c8382890ae
commit
60f3b25d06
|
@ -108,6 +108,10 @@ Optimizations
|
||||||
* LUCENE-8796: Use exponential search instead of binary search in
|
* LUCENE-8796: Use exponential search instead of binary search in
|
||||||
IntArrayDocIdSet#advance method (Luca Cavanna via Adrien Grand)
|
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
|
Test Framework
|
||||||
|
|
||||||
* LUCENE-8825: CheckHits now display the shard index in case of mismatch
|
* LUCENE-8825: CheckHits now display the shard index in case of mismatch
|
||||||
|
|
|
@ -26,7 +26,7 @@ import java.util.Comparator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.Callable;
|
import java.util.concurrent.CompletableFuture;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
import java.util.concurrent.Future;
|
import java.util.concurrent.Future;
|
||||||
|
@ -613,7 +613,7 @@ public class IndexSearcher {
|
||||||
* @lucene.experimental
|
* @lucene.experimental
|
||||||
*/
|
*/
|
||||||
public <C extends Collector, T> T search(Query query, CollectorManager<C, T> collectorManager) throws IOException {
|
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();
|
final C collector = collectorManager.newCollector();
|
||||||
search(query, collector);
|
search(query, collector);
|
||||||
return collectorManager.reduce(Collections.singletonList(collector));
|
return collectorManager.reduce(Collections.singletonList(collector));
|
||||||
|
@ -636,18 +636,19 @@ public class IndexSearcher {
|
||||||
query = rewrite(query);
|
query = rewrite(query);
|
||||||
final Weight weight = createWeight(query, scoreMode, 1);
|
final Weight weight = createWeight(query, scoreMode, 1);
|
||||||
final List<Future<C>> topDocsFutures = new ArrayList<>(leafSlices.length);
|
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 LeafReaderContext[] leaves = leafSlices[i].leaves;
|
||||||
final C collector = collectors.get(i);
|
final C collector = collectors.get(i);
|
||||||
topDocsFutures.add(executor.submit(new Callable<C>() {
|
topDocsFutures.add(executor.submit(() -> {
|
||||||
@Override
|
|
||||||
public C call() throws Exception {
|
|
||||||
search(Arrays.asList(leaves), weight, collector);
|
search(Arrays.asList(leaves), weight, collector);
|
||||||
return 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<>();
|
final List<C> collectedCollectors = new ArrayList<>();
|
||||||
for (Future<C> future : topDocsFutures) {
|
for (Future<C> future : topDocsFutures) {
|
||||||
try {
|
try {
|
||||||
|
@ -658,7 +659,6 @@ public class IndexSearcher {
|
||||||
throw new RuntimeException(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
return collectorManager.reduce(collectors);
|
return collectorManager.reduce(collectors);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue