LUCENE-2440: add custom ExecutorService to ParallelMultiSearcher

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@946338 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael McCandless 2010-05-19 18:58:23 +00:00
parent 9f973b6988
commit 28a153babc
2 changed files with 14 additions and 4 deletions

View File

@ -170,6 +170,10 @@ New features
rewrite payloads of external indexes as they are added, or of local ones).
(Shai Erera, Michael Busch, Mike McCandless)
* LUCENE-2440: Add support for custom ExecutorService in
ParallelMultiSearcher (Edward Drapkin via Mike McCandless)
======================= Lucene 3.x (not yet released) =======================
Changes in backwards compatibility policy

View File

@ -42,19 +42,25 @@ import org.apache.lucene.util.ThreadInterruptedException;
* or {@link #search(Query,Filter,int)} methods.
*/
public class ParallelMultiSearcher extends MultiSearcher {
private final ExecutorService executor;
private final Searchable[] searchables;
private final int[] starts;
/** Creates a {@link Searchable} which searches <i>searchables</i>. */
/** Creates a {@link Searchable} which searches <i>searchables</i> with the default
* executor service (a cached thread pool). */
public ParallelMultiSearcher(Searchable... searchables) throws IOException {
this(Executors.newCachedThreadPool(new NamedThreadFactory(ParallelMultiSearcher.class.getSimpleName())), searchables);
}
/**
* Creates a {@link Searchable} which searches <i>searchables</i> with the specified ExecutorService.
*/
public ParallelMultiSearcher(ExecutorService executor, Searchable... searchables) throws IOException {
super(searchables);
this.searchables = searchables;
this.starts = getStarts();
executor = Executors.newCachedThreadPool(new NamedThreadFactory(this.getClass().getSimpleName()));
this.executor = executor;
}
/**
* Executes each {@link Searchable}'s docFreq() in its own thread and waits for each search to complete and merge
* the results back together.