package org.apache.lucene.search;
/**
* 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.
*/
import java.io.IOException;
import java.util.List;
import java.util.Arrays;
import java.util.ArrayList;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.Callable;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.Term;
import org.apache.lucene.util.NamedThreadFactory;
import org.apache.lucene.util.PriorityQueue;
import org.apache.lucene.util.ThreadInterruptedException;
/** Implements parallel search over a set of Searchables.
*
*
Applications usually need only call the inherited {@link #search(Query,int)}
* 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 searchables. */
public ParallelMultiSearcher(Searchable... searchables) throws IOException {
super(searchables);
this.searchables = searchables;
this.starts = getStarts();
executor = Executors.newCachedThreadPool(new NamedThreadFactory(this.getClass().getSimpleName()));
}
/**
* Executes each {@link Searchable}'s docFreq() in its own thread and waits for each search to complete and merge
* the results back together.
*/
@Override
public int docFreq(final Term term) throws IOException {
@SuppressWarnings("unchecked") final Future {@link Collector#collect(int)} is called for every matching document.
*
* Applications should only use this if they need all of the
* matching documents. The high-level search API ({@link
* Searcher#search(Query,int)}) is usually more efficient, as it skips
* non-high-scoring hits.
*
* This method cannot be parallelized, because {@link Collector}
* supports no concurrent access.
*
* @param weight to match documents
* @param filter if non-null, a bitset used to eliminate some documents
* @param collector to receive hits
*/
@Override
public void search(final Weight weight, final Filter filter, final Collector collector)
throws IOException {
for (int i = 0; i < searchables.length; i++) {
final int start = starts[i];
final Collector hc = new Collector() {
@Override
public void setScorer(final Scorer scorer) throws IOException {
collector.setScorer(scorer);
}
@Override
public void collect(final int doc) throws IOException {
collector.collect(doc);
}
@Override
public void setNextReader(final IndexReader reader, final int docBase) throws IOException {
collector.setNextReader(reader, start + docBase);
}
@Override
public boolean acceptsDocsOutOfOrder() {
return collector.acceptsDocsOutOfOrder();
}
};
searchables[i].search(weight, filter, hc);
}
}
/*
* apply the function to each element of the list. This method encapsulates the logic how
* to wait for concurrently executed searchables.
*/
private