cleanup test and restore realistic test with line file docs

This commit is contained in:
jimczi 2019-10-01 12:44:10 +02:00
parent 7eab6ebb27
commit f55633c881
2 changed files with 164 additions and 79 deletions

View File

@ -31,10 +31,15 @@ import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.MultiTerms;
import org.apache.lucene.index.NoMergePolicy;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LineFileDocs;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.NamedThreadFactory;
@ -132,31 +137,6 @@ public class TestTopDocsCollector extends LuceneTestCase {
}
}
private TopFieldCollector doSearchWithThreshold(int numResults, int thresHold, Query q, Sort sort, IndexReader indexReader) throws IOException {
IndexSearcher searcher = new IndexSearcher(indexReader);
TopFieldCollector tdc = TopFieldCollector.create(sort, numResults, thresHold);
searcher.search(q, tdc);
return tdc;
}
private TopDocs doConcurrentSearchWithThreshold(int numResults, int threshold, Query q, Sort sort, IndexReader indexReader) throws IOException {
ExecutorService service = new ThreadPoolExecutor(4, 4, 0L, TimeUnit.MILLISECONDS,
new LinkedBlockingQueue<Runnable>(),
new NamedThreadFactory("TestTopDocsCollector"));
try {
IndexSearcher searcher = new IndexSearcher(indexReader, service);
CollectorManager collectorManager = TopFieldCollector.createSharedManager(sort, numResults,
null, threshold);
TopDocs tdc = (TopDocs) searcher.search(q, collectorManager);
return tdc;
} finally {
service.shutdown();
}
}
@Override
public void setUp() throws Exception {
super.setUp();
@ -422,60 +402,6 @@ public class TestTopDocsCollector extends LuceneTestCase {
dir.close();
}
public void testGlobalScore() throws Exception {
Directory dir = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random(), dir, newIndexWriterConfig());
int numDocs = atLeast(1000);
for (int i = 0; i < numDocs; ++i) {
int numAs = 1 + random().nextInt(5);
int numBs = random().nextFloat() < 0.5f ? 0 : 1 + random().nextInt(5);
int numCs = random().nextFloat() < 0.1f ? 0 : 1 + random().nextInt(5);
Document doc = new Document();
for (int j = 0; j < numAs; ++j) {
doc.add(new StringField("f", "A", Field.Store.NO));
}
for (int j = 0; j < numBs; ++j) {
doc.add(new StringField("f", "B", Field.Store.NO));
}
for (int j = 0; j < numCs; ++j) {
doc.add(new StringField("f", "C", Field.Store.NO));
}
w.addDocument(doc);
}
IndexReader indexReader = w.getReader();
w.close();
Query[] queries = new Query[]{
new TermQuery(new Term("f", "A")),
new TermQuery(new Term("f", "B")),
new TermQuery(new Term("f", "C")),
new BooleanQuery.Builder()
.add(new TermQuery(new Term("f", "A")), BooleanClause.Occur.MUST)
.add(new TermQuery(new Term("f", "B")), BooleanClause.Occur.SHOULD)
.build()
};
for (Query query : queries) {
TopDocsCollector collector = doSearchWithThreshold(5, 10, query, indexReader);
TopDocs tdc = doConcurrentSearchWithThreshold(5, 10, query, indexReader);
TopDocs tdc2 = collector.topDocs();
assertTrue(tdc.totalHits.value > 0);
assertTrue(tdc2.totalHits.value > 0);
CheckHits.checkEqual(query, tdc.scoreDocs, tdc2.scoreDocs);
Sort sort = new Sort(new SortField[]{SortField.FIELD_SCORE, SortField.FIELD_DOC});
TopDocsCollector fieldCollector = doSearchWithThreshold(5, 10, query, sort, indexReader);
tdc = doConcurrentSearchWithThreshold(5, 10, query, sort, indexReader);
tdc2 = fieldCollector.topDocs();
assertTrue(tdc.totalHits.value > 0);
assertTrue(tdc2.totalHits.value > 0);
CheckHits.checkEqual(query, tdc.scoreDocs, tdc2.scoreDocs);
}
indexReader.close();
dir.close();
}
public void testConcurrentMinScore() throws Exception {
Directory dir = newDirectory();
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig().setMergePolicy(NoMergePolicy.INSTANCE));
@ -591,4 +517,91 @@ public class TestTopDocsCollector extends LuceneTestCase {
reader.close();
dir.close();
}
public void testRandomMinCompetitiveScore() throws Exception {
Directory dir = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random(), dir, newIndexWriterConfig());
int numDocs = atLeast(1000);
for (int i = 0; i < numDocs; ++i) {
int numAs = 1 + random().nextInt(5);
int numBs = random().nextFloat() < 0.5f ? 0 : 1 + random().nextInt(5);
int numCs = random().nextFloat() < 0.1f ? 0 : 1 + random().nextInt(5);
Document doc = new Document();
for (int j = 0; j < numAs; ++j) {
doc.add(new StringField("f", "A", Field.Store.NO));
}
for (int j = 0; j < numBs; ++j) {
doc.add(new StringField("f", "B", Field.Store.NO));
}
for (int j = 0; j < numCs; ++j) {
doc.add(new StringField("f", "C", Field.Store.NO));
}
w.addDocument(doc);
}
IndexReader indexReader = w.getReader();
w.close();
Query[] queries = new Query[]{
new TermQuery(new Term("f", "A")),
new TermQuery(new Term("f", "B")),
new TermQuery(new Term("f", "C")),
new BooleanQuery.Builder()
.add(new TermQuery(new Term("f", "A")), BooleanClause.Occur.MUST)
.add(new TermQuery(new Term("f", "B")), BooleanClause.Occur.SHOULD)
.build()
};
for (Query query : queries) {
TopDocsCollector collector = doSearchWithThreshold(5, 0, query, indexReader);
TopDocs tdc = doConcurrentSearchWithThreshold(5, 0, query, indexReader);
TopDocs tdc2 = collector.topDocs();
assertTrue(tdc.totalHits.value > 0);
assertTrue(tdc2.totalHits.value > 0);
CheckHits.checkEqual(query, tdc.scoreDocs, tdc2.scoreDocs);
}
indexReader.close();
dir.close();
}
public void testRealisticConcurrentMinimumScore() throws Exception {
Directory dir = newDirectory();
RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
try (LineFileDocs docs = new LineFileDocs(random())) {
int numDocs = atLeast(100);
for (int i = 0; i < numDocs; i++) {
writer.addDocument(docs.nextDoc());
}
}
IndexReader reader = writer.getReader();
writer.close();
final IndexSearcher s = newSearcher(reader);
Terms terms = MultiTerms.getTerms(reader, "body");
int termCount = 0;
TermsEnum termsEnum = terms.iterator();
while(termsEnum.next() != null) {
termCount++;
}
assertTrue(termCount > 0);
// Target ~10 terms to search:
double chance = 10.0 / termCount;
termsEnum = terms.iterator();
while(termsEnum.next() != null) {
if (random().nextDouble() <= chance) {
BytesRef term = BytesRef.deepCopyOf(termsEnum.term());
Query query = new TermQuery(new Term("body", term));
TopDocsCollector collector = doSearchWithThreshold(5, 0, query, reader);
TopDocs tdc = doConcurrentSearchWithThreshold(5, 0, query, reader);
TopDocs tdc2 = collector.topDocs();
CheckHits.checkEqual(query, tdc.scoreDocs, tdc2.scoreDocs);
}
}
reader.close();
dir.close();
}
}

View File

@ -26,6 +26,7 @@ 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.document.Field.Store;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.StringField;
@ -73,6 +74,31 @@ public class TestTopFieldCollector extends LuceneTestCase {
super.tearDown();
}
private TopFieldCollector doSearchWithThreshold(int numResults, int thresHold, Query q, Sort sort, IndexReader indexReader) throws IOException {
IndexSearcher searcher = new IndexSearcher(indexReader);
TopFieldCollector tdc = TopFieldCollector.create(sort, numResults, thresHold);
searcher.search(q, tdc);
return tdc;
}
private TopDocs doConcurrentSearchWithThreshold(int numResults, int threshold, Query q, Sort sort, IndexReader indexReader) throws IOException {
ExecutorService service = new ThreadPoolExecutor(4, 4, 0L, TimeUnit.MILLISECONDS,
new LinkedBlockingQueue<Runnable>(),
new NamedThreadFactory("TestTopDocsCollector"));
try {
IndexSearcher searcher = new IndexSearcher(indexReader, service);
CollectorManager collectorManager = TopFieldCollector.createSharedManager(sort, numResults,
null, threshold);
TopDocs tdc = (TopDocs) searcher.search(q, collectorManager);
return tdc;
} finally {
service.shutdown();
}
}
public void testSortWithoutFillFields() throws Exception {
// There was previously a bug in TopFieldCollector when fillFields was set
@ -612,4 +638,50 @@ public class TestTopFieldCollector extends LuceneTestCase {
dir.close();
}
public void testRandomMinCompetitiveScore() throws Exception {
Directory dir = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random(), dir, newIndexWriterConfig());
int numDocs = atLeast(1000);
for (int i = 0; i < numDocs; ++i) {
int numAs = 1 + random().nextInt(5);
int numBs = random().nextFloat() < 0.5f ? 0 : 1 + random().nextInt(5);
int numCs = random().nextFloat() < 0.1f ? 0 : 1 + random().nextInt(5);
Document doc = new Document();
for (int j = 0; j < numAs; ++j) {
doc.add(new StringField("f", "A", Field.Store.NO));
}
for (int j = 0; j < numBs; ++j) {
doc.add(new StringField("f", "B", Field.Store.NO));
}
for (int j = 0; j < numCs; ++j) {
doc.add(new StringField("f", "C", Field.Store.NO));
}
w.addDocument(doc);
}
IndexReader indexReader = w.getReader();
w.close();
Query[] queries = new Query[]{
new TermQuery(new Term("f", "A")),
new TermQuery(new Term("f", "B")),
new TermQuery(new Term("f", "C")),
new BooleanQuery.Builder()
.add(new TermQuery(new Term("f", "A")), BooleanClause.Occur.MUST)
.add(new TermQuery(new Term("f", "B")), BooleanClause.Occur.SHOULD)
.build()
};
for (Query query : queries) {
Sort sort = new Sort(new SortField[]{SortField.FIELD_SCORE, SortField.FIELD_DOC});
TopFieldCollector fieldCollector = doSearchWithThreshold(5, 0, query, sort, indexReader);
TopDocs tdc = doConcurrentSearchWithThreshold(5, 0, query, sort, indexReader);
TopDocs tdc2 = fieldCollector.topDocs();
assertTrue(tdc.totalHits.value > 0);
assertTrue(tdc2.totalHits.value > 0);
CheckHits.checkEqual(query, tdc.scoreDocs, tdc2.scoreDocs);
}
indexReader.close();
dir.close();
}
}