LUCENE-6754: Optimized IndexSearcher.count for simple queries.

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1700791 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Adrien Grand 2015-09-02 12:36:37 +00:00
parent 0baae2f832
commit 5330bd6d2b
4 changed files with 64 additions and 15 deletions

View File

@ -83,6 +83,9 @@ Optimizations
* LUCENE-6746: DisjunctionMaxQuery, BoostingQuery and BoostedQuery now create
sub weights through IndexSearcher so that they can be cached. (Adrien Grand)
* LUCENE-6754: Optimized IndexSearcher.count for the cases when it can use
index statistics instead of collecting all matches. (Adrien Grand)
Bug Fixes
* LUCENE-6730: Hyper-parameter c is ignored in term frequency NormalizationH1.

View File

@ -339,6 +339,29 @@ public class IndexSearcher {
* Count how many documents match the given query.
*/
public int count(Query query) throws IOException {
query = rewrite(query);
while (true) {
// remove wrappers that don't matter for counts
if (query instanceof ConstantScoreQuery) {
query = ((ConstantScoreQuery) query).getQuery();
} else {
break;
}
}
// some counts can be computed in constant time
if (query instanceof MatchAllDocsQuery) {
return reader.numDocs();
} else if (query instanceof TermQuery && reader.hasDeletions() == false) {
Term term = ((TermQuery) query).getTerm();
int count = 0;
for (LeafReaderContext leaf : reader.leaves()) {
count += leaf.reader().docFreq(term);
}
return count;
}
// general case: create a collecor and count matches
final CollectorManager<TotalHitCountCollector, Integer> collectorManager = new CollectorManager<TotalHitCountCollector, Integer>() {
@Override

View File

@ -18,6 +18,7 @@ package org.apache.lucene.search;
*/
import java.io.IOException;
import java.util.Arrays;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
@ -31,6 +32,7 @@ import org.apache.lucene.document.StringField;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
@ -144,15 +146,36 @@ public class TestIndexSearcher extends LuceneTestCase {
if (random().nextBoolean()) {
doc.add(new StringField("foo", "bar", Store.NO));
}
if (random().nextBoolean()) {
doc.add(new StringField("foo", "baz", Store.NO));
}
if (rarely()) {
doc.add(new StringField("delete", "yes", Store.NO));
}
w.addDocument(doc);
}
w.commit();
for (boolean delete : new boolean[] {false, true}) {
if (delete) {
w.deleteDocuments(new Term("delete", "yes"));
}
final IndexReader reader = w.getReader();
w.close();
final IndexSearcher searcher = newSearcher(reader);
final Query query = new TermQuery(new Term("foo", "bar"));
// Test multiple queries, some of them are optimized by IndexSearcher.count()
for (Query query : Arrays.asList(
new MatchAllDocsQuery(),
new MatchNoDocsQuery(),
new TermQuery(new Term("foo", "bar")),
new ConstantScoreQuery(new TermQuery(new Term("foo", "baz"))),
new BooleanQuery.Builder()
.add(new TermQuery(new Term("foo", "bar")), Occur.SHOULD)
.add(new TermQuery(new Term("foo", "baz")), Occur.SHOULD)
.build()
)) {
assertEquals(searcher.count(query), searcher.search(query, 1).totalHits);
}
reader.close();
}
w.close();
dir.close();
}

View File

@ -971,7 +971,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
try {
// trigger an eviction
searcher.count(new MatchAllDocsQuery());
searcher.search(new MatchAllDocsQuery(), new TotalHitCountCollector());
fail();
} catch (ConcurrentModificationException e) {
// expected