mirror of https://github.com/apache/lucene.git
LUCENE-4938: IndexSearcher.search() with sort doesnt do min(maxdoc, n)
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1469889 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
822cb29ec4
commit
f98ac76130
|
@ -42,6 +42,11 @@ Bug Fixes
|
|||
* LUCENE-4935: CustomScoreQuery wrongly applied its query boost twice
|
||||
(boost^2). (Robert Muir)
|
||||
|
||||
Optimizations
|
||||
|
||||
* LUCENE-4938: Don't use an unnecessarily large priority queue in IndexSearcher
|
||||
methods that take top-N. (Uwe Schindler, Mike McCandless, Robert Muir)
|
||||
|
||||
======================= Lucene 4.3.0 =======================
|
||||
|
||||
Changes in backwards compatibility policy
|
||||
|
|
|
@ -426,6 +426,12 @@ public class IndexSearcher {
|
|||
* {@link BooleanQuery#getMaxClauseCount()} clauses.
|
||||
*/
|
||||
protected TopDocs search(Weight weight, ScoreDoc after, int nDocs) throws IOException {
|
||||
int limit = reader.maxDoc();
|
||||
if (limit == 0) {
|
||||
limit = 1;
|
||||
}
|
||||
nDocs = Math.min(nDocs, limit);
|
||||
|
||||
if (executor == null) {
|
||||
return search(leafContexts, weight, after, nDocs);
|
||||
} else {
|
||||
|
@ -511,6 +517,12 @@ public class IndexSearcher {
|
|||
|
||||
if (sort == null) throw new NullPointerException("Sort must not be null");
|
||||
|
||||
int limit = reader.maxDoc();
|
||||
if (limit == 0) {
|
||||
limit = 1;
|
||||
}
|
||||
nDocs = Math.min(nDocs, limit);
|
||||
|
||||
if (executor == null) {
|
||||
// use all leaves here!
|
||||
return search(leafContexts, weight, after, nDocs, sort, fillFields, doDocScores, doMaxScore);
|
||||
|
|
|
@ -0,0 +1,119 @@
|
|||
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.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.NamedThreadFactory;
|
||||
import org.apache.lucene.util._TestUtil;
|
||||
|
||||
public class TestIndexSearcher extends LuceneTestCase {
|
||||
Directory dir;
|
||||
IndexReader reader;
|
||||
|
||||
@Override
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
dir = newDirectory();
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
|
||||
for (int i = 0; i < 100; i++) {
|
||||
Document doc = new Document();
|
||||
doc.add(newStringField("field", Integer.toString(i), Field.Store.NO));
|
||||
doc.add(newStringField("field2", Boolean.toString(i % 2 == 0), Field.Store.NO));
|
||||
iw.addDocument(doc);
|
||||
}
|
||||
reader = iw.getReader();
|
||||
iw.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void tearDown() throws Exception {
|
||||
super.tearDown();
|
||||
reader.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
// should not throw exception
|
||||
public void testHugeN() throws Exception {
|
||||
ExecutorService service = new ThreadPoolExecutor(4, 4, 0L, TimeUnit.MILLISECONDS,
|
||||
new LinkedBlockingQueue<Runnable>(),
|
||||
new NamedThreadFactory("TestIndexSearcher"));
|
||||
|
||||
IndexSearcher searchers[] = new IndexSearcher[] {
|
||||
new IndexSearcher(reader),
|
||||
new IndexSearcher(reader, service)
|
||||
};
|
||||
Query queries[] = new Query[] {
|
||||
new MatchAllDocsQuery(),
|
||||
new TermQuery(new Term("field", "1"))
|
||||
};
|
||||
Sort sorts[] = new Sort[] {
|
||||
null,
|
||||
new Sort(new SortField("field2", SortField.Type.STRING))
|
||||
};
|
||||
Filter filters[] = new Filter[] {
|
||||
null,
|
||||
new QueryWrapperFilter(new TermQuery(new Term("field2", "true")))
|
||||
};
|
||||
ScoreDoc afters[] = new ScoreDoc[] {
|
||||
null,
|
||||
new FieldDoc(0, 0f, new Object[] { new BytesRef("boo!") })
|
||||
};
|
||||
|
||||
for (IndexSearcher searcher : searchers) {
|
||||
for (ScoreDoc after : afters) {
|
||||
for (Query query : queries) {
|
||||
for (Sort sort : sorts) {
|
||||
for (Filter filter : filters) {
|
||||
searcher.search(query, Integer.MAX_VALUE);
|
||||
searcher.searchAfter(after, query, Integer.MAX_VALUE);
|
||||
searcher.search(query, filter, Integer.MAX_VALUE);
|
||||
searcher.searchAfter(after, query, filter, Integer.MAX_VALUE);
|
||||
if (sort != null) {
|
||||
searcher.search(query, Integer.MAX_VALUE, sort);
|
||||
searcher.search(query, filter, Integer.MAX_VALUE, sort);
|
||||
searcher.search(query, filter, Integer.MAX_VALUE, sort, true, true);
|
||||
searcher.search(query, filter, Integer.MAX_VALUE, sort, true, false);
|
||||
searcher.search(query, filter, Integer.MAX_VALUE, sort, false, true);
|
||||
searcher.search(query, filter, Integer.MAX_VALUE, sort, false, false);
|
||||
searcher.searchAfter(after, query, filter, Integer.MAX_VALUE, sort);
|
||||
searcher.searchAfter(after, query, filter, Integer.MAX_VALUE, sort, true, true);
|
||||
searcher.searchAfter(after, query, filter, Integer.MAX_VALUE, sort, true, false);
|
||||
searcher.searchAfter(after, query, filter, Integer.MAX_VALUE, sort, false, true);
|
||||
searcher.searchAfter(after, query, filter, Integer.MAX_VALUE, sort, false, false);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
_TestUtil.shutdownExecutorService(service);
|
||||
}
|
||||
}
|
|
@ -69,7 +69,7 @@ public class TestFunctionQuerySort extends LuceneTestCase {
|
|||
|
||||
// Get hits sorted by our FunctionValues (ascending values)
|
||||
Query q = new MatchAllDocsQuery();
|
||||
TopDocs hits = searcher.search(q, Integer.MAX_VALUE, orderBy);
|
||||
TopDocs hits = searcher.search(q, reader.maxDoc(), orderBy);
|
||||
assertEquals(NUM_VALS, hits.scoreDocs.length);
|
||||
// Verify that sorting works in general
|
||||
int i = 0;
|
||||
|
@ -81,7 +81,7 @@ public class TestFunctionQuerySort extends LuceneTestCase {
|
|||
// Now get hits after hit #2 using IS.searchAfter()
|
||||
int afterIdx = 1;
|
||||
FieldDoc afterHit = (FieldDoc) hits.scoreDocs[afterIdx];
|
||||
hits = searcher.searchAfter(afterHit, q, Integer.MAX_VALUE, orderBy);
|
||||
hits = searcher.searchAfter(afterHit, q, reader.maxDoc(), orderBy);
|
||||
|
||||
// Expected # of hits: NUM_VALS - 2
|
||||
assertEquals(NUM_VALS - (afterIdx + 1), hits.scoreDocs.length);
|
||||
|
|
Loading…
Reference in New Issue