mirror of https://github.com/apache/lucene.git
LUCENE-3053: improve test coverage for Multi*
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1098303 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
a47d4b835b
commit
d622e54172
|
@ -36,8 +36,8 @@ public class TestFieldCacheRewriteMethod extends TestRegexpRandom2 {
|
||||||
RegexpQuery filter = new RegexpQuery(new Term("field", regexp), RegExp.NONE);
|
RegexpQuery filter = new RegexpQuery(new Term("field", regexp), RegExp.NONE);
|
||||||
filter.setRewriteMethod(MultiTermQuery.CONSTANT_SCORE_FILTER_REWRITE);
|
filter.setRewriteMethod(MultiTermQuery.CONSTANT_SCORE_FILTER_REWRITE);
|
||||||
|
|
||||||
TopDocs fieldCacheDocs = searcher.search(fieldCache, 25);
|
TopDocs fieldCacheDocs = searcher1.search(fieldCache, 25);
|
||||||
TopDocs filterDocs = searcher.search(filter, 25);
|
TopDocs filterDocs = searcher2.search(filter, 25);
|
||||||
|
|
||||||
CheckHits.checkEqual(fieldCache, fieldCacheDocs.scoreDocs, filterDocs.scoreDocs);
|
CheckHits.checkEqual(fieldCache, fieldCacheDocs.scoreDocs, filterDocs.scoreDocs);
|
||||||
}
|
}
|
||||||
|
|
|
@ -187,7 +187,8 @@ public class TestParser extends LuceneTestCase {
|
||||||
}
|
}
|
||||||
public void testDuplicateFilterQueryXML() throws ParserException, IOException
|
public void testDuplicateFilterQueryXML() throws ParserException, IOException
|
||||||
{
|
{
|
||||||
Assume.assumeTrue(searcher.getIndexReader().getSequentialSubReaders().length == 1);
|
Assume.assumeTrue(searcher.getIndexReader().getSequentialSubReaders() == null ||
|
||||||
|
searcher.getIndexReader().getSequentialSubReaders().length == 1);
|
||||||
Query q=parse("DuplicateFilterQuery.xml");
|
Query q=parse("DuplicateFilterQuery.xml");
|
||||||
int h = searcher.search(q, null, 1000).totalHits;
|
int h = searcher.search(q, null, 1000).totalHits;
|
||||||
assertEquals("DuplicateFilterQuery should produce 1 result ", 1,h);
|
assertEquals("DuplicateFilterQuery should produce 1 result ", 1,h);
|
||||||
|
|
|
@ -309,7 +309,7 @@ public class QueryUtils {
|
||||||
// confirm that skipping beyond the last doc, on the
|
// confirm that skipping beyond the last doc, on the
|
||||||
// previous reader, hits NO_MORE_DOCS
|
// previous reader, hits NO_MORE_DOCS
|
||||||
final IndexReader previousReader = lastReader[0];
|
final IndexReader previousReader = lastReader[0];
|
||||||
IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader);
|
IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader, false);
|
||||||
Weight w = q.weight(indexSearcher);
|
Weight w = q.weight(indexSearcher);
|
||||||
Scorer scorer = w.scorer((AtomicReaderContext)previousReader.getTopReaderContext(), ScorerContext.def());
|
Scorer scorer = w.scorer((AtomicReaderContext)previousReader.getTopReaderContext(), ScorerContext.def());
|
||||||
if (scorer != null) {
|
if (scorer != null) {
|
||||||
|
|
|
@ -1091,8 +1091,21 @@ public abstract class LuceneTestCase extends Assert {
|
||||||
/** create a new searcher over the reader.
|
/** create a new searcher over the reader.
|
||||||
* This searcher might randomly use threads. */
|
* This searcher might randomly use threads. */
|
||||||
public static IndexSearcher newSearcher(IndexReader r) throws IOException {
|
public static IndexSearcher newSearcher(IndexReader r) throws IOException {
|
||||||
|
return newSearcher(r, true);
|
||||||
|
}
|
||||||
|
|
||||||
|
/** create a new searcher over the reader.
|
||||||
|
* This searcher might randomly use threads.
|
||||||
|
* if <code>maybeWrap</code> is true, this searcher might wrap the reader
|
||||||
|
* with one that returns null for getSequentialSubReaders.
|
||||||
|
*/
|
||||||
|
public static IndexSearcher newSearcher(IndexReader r, boolean maybeWrap) throws IOException {
|
||||||
if (random.nextBoolean()) {
|
if (random.nextBoolean()) {
|
||||||
|
if (maybeWrap && random.nextBoolean()) {
|
||||||
|
return new IndexSearcher(new SlowMultiReaderWrapper(r));
|
||||||
|
} else {
|
||||||
return new IndexSearcher(r);
|
return new IndexSearcher(r);
|
||||||
|
}
|
||||||
} else {
|
} else {
|
||||||
int threads = 0;
|
int threads = 0;
|
||||||
final ExecutorService ex = (random.nextBoolean()) ? null
|
final ExecutorService ex = (random.nextBoolean()) ? null
|
||||||
|
|
|
@ -49,7 +49,8 @@ public class TestCachingSpanFilter extends LuceneTestCase {
|
||||||
// but we use .reopen on this reader below and expect to
|
// but we use .reopen on this reader below and expect to
|
||||||
// (must) get an NRT reader:
|
// (must) get an NRT reader:
|
||||||
IndexReader reader = IndexReader.open(writer.w, true);
|
IndexReader reader = IndexReader.open(writer.w, true);
|
||||||
IndexSearcher searcher = newSearcher(reader);
|
// same reason we don't wrap?
|
||||||
|
IndexSearcher searcher = newSearcher(reader, false);
|
||||||
|
|
||||||
// add a doc, refresh the reader, and check that its there
|
// add a doc, refresh the reader, and check that its there
|
||||||
Document doc = new Document();
|
Document doc = new Document();
|
||||||
|
@ -58,7 +59,7 @@ public class TestCachingSpanFilter extends LuceneTestCase {
|
||||||
|
|
||||||
reader = refreshReader(reader);
|
reader = refreshReader(reader);
|
||||||
searcher.close();
|
searcher.close();
|
||||||
searcher = newSearcher(reader);
|
searcher = newSearcher(reader, false);
|
||||||
|
|
||||||
TopDocs docs = searcher.search(new MatchAllDocsQuery(), 1);
|
TopDocs docs = searcher.search(new MatchAllDocsQuery(), 1);
|
||||||
assertEquals("Should find a hit...", 1, docs.totalHits);
|
assertEquals("Should find a hit...", 1, docs.totalHits);
|
||||||
|
@ -81,7 +82,7 @@ public class TestCachingSpanFilter extends LuceneTestCase {
|
||||||
|
|
||||||
reader = refreshReader(reader);
|
reader = refreshReader(reader);
|
||||||
searcher.close();
|
searcher.close();
|
||||||
searcher = newSearcher(reader);
|
searcher = newSearcher(reader, false);
|
||||||
|
|
||||||
docs = searcher.search(new MatchAllDocsQuery(), filter, 1);
|
docs = searcher.search(new MatchAllDocsQuery(), filter, 1);
|
||||||
assertEquals("[query + filter] Should *not* find a hit...", 0, docs.totalHits);
|
assertEquals("[query + filter] Should *not* find a hit...", 0, docs.totalHits);
|
||||||
|
@ -96,7 +97,7 @@ public class TestCachingSpanFilter extends LuceneTestCase {
|
||||||
writer.addDocument(doc);
|
writer.addDocument(doc);
|
||||||
reader = refreshReader(reader);
|
reader = refreshReader(reader);
|
||||||
searcher.close();
|
searcher.close();
|
||||||
searcher = newSearcher(reader);
|
searcher = newSearcher(reader, false);
|
||||||
|
|
||||||
docs = searcher.search(new MatchAllDocsQuery(), filter, 1);
|
docs = searcher.search(new MatchAllDocsQuery(), filter, 1);
|
||||||
assertEquals("[query + filter] Should find a hit...", 1, docs.totalHits);
|
assertEquals("[query + filter] Should find a hit...", 1, docs.totalHits);
|
||||||
|
@ -115,7 +116,7 @@ public class TestCachingSpanFilter extends LuceneTestCase {
|
||||||
reader = refreshReader(reader);
|
reader = refreshReader(reader);
|
||||||
assertTrue(reader != oldReader);
|
assertTrue(reader != oldReader);
|
||||||
searcher.close();
|
searcher.close();
|
||||||
searcher = newSearcher(reader);
|
searcher = newSearcher(reader, false);
|
||||||
int missCount = filter.missCount;
|
int missCount = filter.missCount;
|
||||||
docs = searcher.search(constantScore, 1);
|
docs = searcher.search(constantScore, 1);
|
||||||
assertEquals("[just filter] Should find a hit...", 1, docs.totalHits);
|
assertEquals("[just filter] Should find a hit...", 1, docs.totalHits);
|
||||||
|
@ -126,7 +127,7 @@ public class TestCachingSpanFilter extends LuceneTestCase {
|
||||||
|
|
||||||
reader = refreshReader(reader);
|
reader = refreshReader(reader);
|
||||||
searcher.close();
|
searcher.close();
|
||||||
searcher = newSearcher(reader);
|
searcher = newSearcher(reader, false);
|
||||||
|
|
||||||
docs = searcher.search(new MatchAllDocsQuery(), filter, 1);
|
docs = searcher.search(new MatchAllDocsQuery(), filter, 1);
|
||||||
assertEquals("[query + filter] Should *not* find a hit...", 0, docs.totalHits);
|
assertEquals("[query + filter] Should *not* find a hit...", 0, docs.totalHits);
|
||||||
|
|
|
@ -171,7 +171,8 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
|
||||||
// but we use .reopen on this reader below and expect to
|
// but we use .reopen on this reader below and expect to
|
||||||
// (must) get an NRT reader:
|
// (must) get an NRT reader:
|
||||||
IndexReader reader = IndexReader.open(writer.w, true);
|
IndexReader reader = IndexReader.open(writer.w, true);
|
||||||
IndexSearcher searcher = newSearcher(reader);
|
// same reason we don't wrap?
|
||||||
|
IndexSearcher searcher = newSearcher(reader, false);
|
||||||
|
|
||||||
// add a doc, refresh the reader, and check that its there
|
// add a doc, refresh the reader, and check that its there
|
||||||
Document doc = new Document();
|
Document doc = new Document();
|
||||||
|
@ -180,7 +181,7 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
|
||||||
|
|
||||||
reader = refreshReader(reader);
|
reader = refreshReader(reader);
|
||||||
searcher.close();
|
searcher.close();
|
||||||
searcher = newSearcher(reader);
|
searcher = newSearcher(reader, false);
|
||||||
|
|
||||||
TopDocs docs = searcher.search(new MatchAllDocsQuery(), 1);
|
TopDocs docs = searcher.search(new MatchAllDocsQuery(), 1);
|
||||||
assertEquals("Should find a hit...", 1, docs.totalHits);
|
assertEquals("Should find a hit...", 1, docs.totalHits);
|
||||||
|
@ -202,7 +203,7 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
|
||||||
|
|
||||||
reader = refreshReader(reader);
|
reader = refreshReader(reader);
|
||||||
searcher.close();
|
searcher.close();
|
||||||
searcher = newSearcher(reader);
|
searcher = newSearcher(reader, false);
|
||||||
|
|
||||||
docs = searcher.search(new MatchAllDocsQuery(), filter, 1);
|
docs = searcher.search(new MatchAllDocsQuery(), filter, 1);
|
||||||
assertEquals("[query + filter] Should *not* find a hit...", 0, docs.totalHits);
|
assertEquals("[query + filter] Should *not* find a hit...", 0, docs.totalHits);
|
||||||
|
@ -218,7 +219,7 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
|
||||||
|
|
||||||
reader = refreshReader(reader);
|
reader = refreshReader(reader);
|
||||||
searcher.close();
|
searcher.close();
|
||||||
searcher = newSearcher(reader);
|
searcher = newSearcher(reader, false);
|
||||||
|
|
||||||
docs = searcher.search(new MatchAllDocsQuery(), filter, 1);
|
docs = searcher.search(new MatchAllDocsQuery(), filter, 1);
|
||||||
|
|
||||||
|
@ -238,7 +239,7 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
|
||||||
reader = refreshReader(reader);
|
reader = refreshReader(reader);
|
||||||
assertTrue(reader != oldReader);
|
assertTrue(reader != oldReader);
|
||||||
searcher.close();
|
searcher.close();
|
||||||
searcher = newSearcher(reader);
|
searcher = newSearcher(reader, false);
|
||||||
int missCount = filter.missCount;
|
int missCount = filter.missCount;
|
||||||
docs = searcher.search(constantScore, 1);
|
docs = searcher.search(constantScore, 1);
|
||||||
assertEquals("[just filter] Should find a hit...", 1, docs.totalHits);
|
assertEquals("[just filter] Should find a hit...", 1, docs.totalHits);
|
||||||
|
@ -249,7 +250,7 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
|
||||||
|
|
||||||
reader = refreshReader(reader);
|
reader = refreshReader(reader);
|
||||||
searcher.close();
|
searcher.close();
|
||||||
searcher = newSearcher(reader);
|
searcher = newSearcher(reader, false);
|
||||||
|
|
||||||
missCount = filter.missCount;
|
missCount = filter.missCount;
|
||||||
docs = searcher.search(new MatchAllDocsQuery(), filter, 1);
|
docs = searcher.search(new MatchAllDocsQuery(), filter, 1);
|
||||||
|
@ -265,7 +266,7 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
|
||||||
writer.addDocument(doc);
|
writer.addDocument(doc);
|
||||||
reader = refreshReader(reader);
|
reader = refreshReader(reader);
|
||||||
searcher.close();
|
searcher.close();
|
||||||
searcher = newSearcher(reader);
|
searcher = newSearcher(reader, false);
|
||||||
|
|
||||||
docs = searcher.search(new MatchAllDocsQuery(), filter, 1);
|
docs = searcher.search(new MatchAllDocsQuery(), filter, 1);
|
||||||
assertEquals("[query + filter] Should find a hit...", 1, docs.totalHits);
|
assertEquals("[query + filter] Should find a hit...", 1, docs.totalHits);
|
||||||
|
@ -278,7 +279,7 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
|
||||||
|
|
||||||
reader = refreshReader(reader);
|
reader = refreshReader(reader);
|
||||||
searcher.close();
|
searcher.close();
|
||||||
searcher = newSearcher(reader);
|
searcher = newSearcher(reader, false);
|
||||||
|
|
||||||
docs = searcher.search(new MatchAllDocsQuery(), filter, 1);
|
docs = searcher.search(new MatchAllDocsQuery(), filter, 1);
|
||||||
assertEquals("[query + filter] Should *not* find a hit...", 0, docs.totalHits);
|
assertEquals("[query + filter] Should *not* find a hit...", 0, docs.totalHits);
|
||||||
|
|
|
@ -54,9 +54,9 @@ public class TestMatchAllDocsQuery extends LuceneTestCase {
|
||||||
|
|
||||||
hits = is.search(new MatchAllDocsQuery(), null, 1000).scoreDocs;
|
hits = is.search(new MatchAllDocsQuery(), null, 1000).scoreDocs;
|
||||||
assertEquals(3, hits.length);
|
assertEquals(3, hits.length);
|
||||||
assertEquals("one", ir.document(hits[0].doc).get("key"));
|
assertEquals("one", is.doc(hits[0].doc).get("key"));
|
||||||
assertEquals("two", ir.document(hits[1].doc).get("key"));
|
assertEquals("two", is.doc(hits[1].doc).get("key"));
|
||||||
assertEquals("three four", ir.document(hits[2].doc).get("key"));
|
assertEquals("three four", is.doc(hits[2].doc).get("key"));
|
||||||
|
|
||||||
// assert with norms scoring turned on
|
// assert with norms scoring turned on
|
||||||
|
|
||||||
|
@ -64,19 +64,19 @@ public class TestMatchAllDocsQuery extends LuceneTestCase {
|
||||||
hits = is.search(normsQuery, null, 1000).scoreDocs;
|
hits = is.search(normsQuery, null, 1000).scoreDocs;
|
||||||
assertEquals(3, hits.length);
|
assertEquals(3, hits.length);
|
||||||
|
|
||||||
assertEquals("three four", ir.document(hits[0].doc).get("key"));
|
assertEquals("three four", is.doc(hits[0].doc).get("key"));
|
||||||
assertEquals("two", ir.document(hits[1].doc).get("key"));
|
assertEquals("two", is.doc(hits[1].doc).get("key"));
|
||||||
assertEquals("one", ir.document(hits[2].doc).get("key"));
|
assertEquals("one", is.doc(hits[2].doc).get("key"));
|
||||||
|
|
||||||
// change norm & retest
|
// change norm & retest
|
||||||
ir.setNorm(0, "key", is.getSimilarityProvider().get("key").encodeNormValue(400f));
|
is.getIndexReader().setNorm(0, "key", is.getSimilarityProvider().get("key").encodeNormValue(400f));
|
||||||
normsQuery = new MatchAllDocsQuery("key");
|
normsQuery = new MatchAllDocsQuery("key");
|
||||||
hits = is.search(normsQuery, null, 1000).scoreDocs;
|
hits = is.search(normsQuery, null, 1000).scoreDocs;
|
||||||
assertEquals(3, hits.length);
|
assertEquals(3, hits.length);
|
||||||
|
|
||||||
assertEquals("one", ir.document(hits[0].doc).get("key"));
|
assertEquals("one", is.doc(hits[0].doc).get("key"));
|
||||||
assertEquals("three four", ir.document(hits[1].doc).get("key"));
|
assertEquals("three four", is.doc(hits[1].doc).get("key"));
|
||||||
assertEquals("two", ir.document(hits[2].doc).get("key"));
|
assertEquals("two", is.doc(hits[2].doc).get("key"));
|
||||||
|
|
||||||
// some artificial queries to trigger the use of skipTo():
|
// some artificial queries to trigger the use of skipTo():
|
||||||
|
|
||||||
|
@ -93,7 +93,7 @@ public class TestMatchAllDocsQuery extends LuceneTestCase {
|
||||||
assertEquals(1, hits.length);
|
assertEquals(1, hits.length);
|
||||||
|
|
||||||
// delete a document:
|
// delete a document:
|
||||||
ir.deleteDocument(0);
|
is.getIndexReader().deleteDocument(0);
|
||||||
hits = is.search(new MatchAllDocsQuery(), null, 1000).scoreDocs;
|
hits = is.search(new MatchAllDocsQuery(), null, 1000).scoreDocs;
|
||||||
assertEquals(2, hits.length);
|
assertEquals(2, hits.length);
|
||||||
|
|
||||||
|
|
|
@ -154,7 +154,9 @@ public class TestNumericRangeQuery32 extends LuceneTestCase {
|
||||||
assertEquals("First doc"+type, 2*distance+startOffset, Integer.parseInt(doc.get(field)) );
|
assertEquals("First doc"+type, 2*distance+startOffset, Integer.parseInt(doc.get(field)) );
|
||||||
doc=searcher.doc(sd[sd.length-1].doc);
|
doc=searcher.doc(sd[sd.length-1].doc);
|
||||||
assertEquals("Last doc"+type, (1+count)*distance+startOffset, Integer.parseInt(doc.get(field)) );
|
assertEquals("Last doc"+type, (1+count)*distance+startOffset, Integer.parseInt(doc.get(field)) );
|
||||||
if (i>0 && searcher.getIndexReader().getSequentialSubReaders().length == 1) {
|
if (i>0 &&
|
||||||
|
(searcher.getIndexReader().getSequentialSubReaders() == null ||
|
||||||
|
searcher.getIndexReader().getSequentialSubReaders().length == 1)) {
|
||||||
assertEquals("Distinct term number is equal for all query types", lastTerms, terms);
|
assertEquals("Distinct term number is equal for all query types", lastTerms, terms);
|
||||||
}
|
}
|
||||||
lastTerms = terms;
|
lastTerms = terms;
|
||||||
|
@ -378,7 +380,9 @@ public class TestNumericRangeQuery32 extends LuceneTestCase {
|
||||||
termCountT += tq.getTotalNumberOfTerms();
|
termCountT += tq.getTotalNumberOfTerms();
|
||||||
termCountC += cq.getTotalNumberOfTerms();
|
termCountC += cq.getTotalNumberOfTerms();
|
||||||
}
|
}
|
||||||
if (precisionStep == Integer.MAX_VALUE && searcher.getIndexReader().getSequentialSubReaders().length == 1) {
|
if (precisionStep == Integer.MAX_VALUE &&
|
||||||
|
(searcher.getIndexReader().getSequentialSubReaders() == null ||
|
||||||
|
searcher.getIndexReader().getSequentialSubReaders().length == 1)) {
|
||||||
assertEquals("Total number of terms should be equal for unlimited precStep", termCountT, termCountC);
|
assertEquals("Total number of terms should be equal for unlimited precStep", termCountT, termCountC);
|
||||||
} else if (VERBOSE) {
|
} else if (VERBOSE) {
|
||||||
System.out.println("Average number of terms during random search on '" + field + "':");
|
System.out.println("Average number of terms during random search on '" + field + "':");
|
||||||
|
|
|
@ -154,7 +154,9 @@ public class TestNumericRangeQuery64 extends LuceneTestCase {
|
||||||
assertEquals("First doc"+type, 2*distance+startOffset, Long.parseLong(doc.get(field)) );
|
assertEquals("First doc"+type, 2*distance+startOffset, Long.parseLong(doc.get(field)) );
|
||||||
doc=searcher.doc(sd[sd.length-1].doc);
|
doc=searcher.doc(sd[sd.length-1].doc);
|
||||||
assertEquals("Last doc"+type, (1+count)*distance+startOffset, Long.parseLong(doc.get(field)) );
|
assertEquals("Last doc"+type, (1+count)*distance+startOffset, Long.parseLong(doc.get(field)) );
|
||||||
if (i>0 && searcher.getIndexReader().getSequentialSubReaders().length == 1) {
|
if (i>0 &&
|
||||||
|
(searcher.getIndexReader().getSequentialSubReaders() == null ||
|
||||||
|
searcher.getIndexReader().getSequentialSubReaders().length == 1)) {
|
||||||
assertEquals("Distinct term number is equal for all query types", lastTerms, terms);
|
assertEquals("Distinct term number is equal for all query types", lastTerms, terms);
|
||||||
}
|
}
|
||||||
lastTerms = terms;
|
lastTerms = terms;
|
||||||
|
@ -395,7 +397,9 @@ public class TestNumericRangeQuery64 extends LuceneTestCase {
|
||||||
termCountT += tq.getTotalNumberOfTerms();
|
termCountT += tq.getTotalNumberOfTerms();
|
||||||
termCountC += cq.getTotalNumberOfTerms();
|
termCountC += cq.getTotalNumberOfTerms();
|
||||||
}
|
}
|
||||||
if (precisionStep == Integer.MAX_VALUE && searcher.getIndexReader().getSequentialSubReaders().length == 1) {
|
if (precisionStep == Integer.MAX_VALUE &&
|
||||||
|
(searcher.getIndexReader().getSequentialSubReaders() == null ||
|
||||||
|
searcher.getIndexReader().getSequentialSubReaders().length == 1)) {
|
||||||
assertEquals("Total number of terms should be equal for unlimited precStep", termCountT, termCountC);
|
assertEquals("Total number of terms should be equal for unlimited precStep", termCountT, termCountC);
|
||||||
} else if (VERBOSE) {
|
} else if (VERBOSE) {
|
||||||
System.out.println("Average number of terms during random search on '" + field + "':");
|
System.out.println("Average number of terms during random search on '" + field + "':");
|
||||||
|
|
|
@ -49,7 +49,8 @@ import org.apache.lucene.util.automaton.RegExp;
|
||||||
* Generates random regexps, and validates against a simple impl.
|
* Generates random regexps, and validates against a simple impl.
|
||||||
*/
|
*/
|
||||||
public class TestRegexpRandom2 extends LuceneTestCase {
|
public class TestRegexpRandom2 extends LuceneTestCase {
|
||||||
protected IndexSearcher searcher;
|
protected IndexSearcher searcher1;
|
||||||
|
protected IndexSearcher searcher2;
|
||||||
private IndexReader reader;
|
private IndexReader reader;
|
||||||
private Directory dir;
|
private Directory dir;
|
||||||
|
|
||||||
|
@ -82,14 +83,16 @@ public class TestRegexpRandom2 extends LuceneTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
reader = writer.getReader();
|
reader = writer.getReader();
|
||||||
searcher = newSearcher(reader);
|
searcher1 = newSearcher(reader);
|
||||||
|
searcher2 = newSearcher(reader);
|
||||||
writer.close();
|
writer.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void tearDown() throws Exception {
|
public void tearDown() throws Exception {
|
||||||
reader.close();
|
reader.close();
|
||||||
searcher.close();
|
searcher1.close();
|
||||||
|
searcher2.close();
|
||||||
dir.close();
|
dir.close();
|
||||||
super.tearDown();
|
super.tearDown();
|
||||||
}
|
}
|
||||||
|
@ -157,12 +160,12 @@ public class TestRegexpRandom2 extends LuceneTestCase {
|
||||||
// automatically comparable.
|
// automatically comparable.
|
||||||
|
|
||||||
// TODO: does this check even matter anymore?!
|
// TODO: does this check even matter anymore?!
|
||||||
Terms terms = MultiFields.getTerms(searcher.getIndexReader(), "field");
|
Terms terms = MultiFields.getTerms(searcher1.getIndexReader(), "field");
|
||||||
if (!(smart.getTermsEnum(terms) instanceof AutomatonTermsEnum))
|
if (!(smart.getTermsEnum(terms) instanceof AutomatonTermsEnum))
|
||||||
return;
|
return;
|
||||||
|
|
||||||
TopDocs smartDocs = searcher.search(smart, 25);
|
TopDocs smartDocs = searcher1.search(smart, 25);
|
||||||
TopDocs dumbDocs = searcher.search(dumb, 25);
|
TopDocs dumbDocs = searcher2.search(dumb, 25);
|
||||||
|
|
||||||
CheckHits.checkEqual(smart, smartDocs.scoreDocs, dumbDocs.scoreDocs);
|
CheckHits.checkEqual(smart, smartDocs.scoreDocs, dumbDocs.scoreDocs);
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue