mirror of
https://github.com/apache/lucene.git
synced 2025-02-28 13:29:26 +00:00
LUCENE-7326: don't use postings format by name in this test
This commit is contained in:
parent
0eacfe87c2
commit
816fae9622
@ -16,13 +16,6 @@
|
||||
*/
|
||||
package org.apache.lucene.index;
|
||||
|
||||
import static org.apache.lucene.index.PostingsEnum.ALL;
|
||||
import static org.apache.lucene.index.PostingsEnum.FREQS;
|
||||
import static org.apache.lucene.index.PostingsEnum.NONE;
|
||||
import static org.apache.lucene.index.PostingsEnum.OFFSETS;
|
||||
import static org.apache.lucene.index.PostingsEnum.PAYLOADS;
|
||||
import static org.apache.lucene.index.PostingsEnum.POSITIONS;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Path;
|
||||
import java.util.Collections;
|
||||
@ -38,9 +31,8 @@ import org.apache.lucene.analysis.MockTokenizer;
|
||||
import org.apache.lucene.analysis.Token;
|
||||
import org.apache.lucene.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.codecs.FieldsProducer;
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.asserting.AssertingCodec;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
@ -51,13 +43,19 @@ import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.LineFileDocs;
|
||||
import org.apache.lucene.util.RamUsageTester;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
|
||||
import static org.apache.lucene.index.PostingsEnum.ALL;
|
||||
import static org.apache.lucene.index.PostingsEnum.FREQS;
|
||||
import static org.apache.lucene.index.PostingsEnum.NONE;
|
||||
import static org.apache.lucene.index.PostingsEnum.OFFSETS;
|
||||
import static org.apache.lucene.index.PostingsEnum.PAYLOADS;
|
||||
import static org.apache.lucene.index.PostingsEnum.POSITIONS;
|
||||
|
||||
/**
|
||||
* Abstract class to do basic tests for a postings format.
|
||||
* NOTE: This test focuses on the postings
|
||||
@ -165,8 +163,9 @@ public abstract class BasePostingsFormatTestCase extends BaseIndexFileFormatTest
|
||||
|
||||
te.seekExact(fieldAndTerm.term);
|
||||
checkReuse(te, PostingsEnum.FREQS, PostingsEnum.ALL, false);
|
||||
if (isPostingsEnumReuseImplemented())
|
||||
if (isPostingsEnumReuseImplemented()) {
|
||||
checkReuse(te, PostingsEnum.ALL, PostingsEnum.ALL, true);
|
||||
}
|
||||
|
||||
fieldsProducer.close();
|
||||
dir.close();
|
||||
@ -175,10 +174,11 @@ public abstract class BasePostingsFormatTestCase extends BaseIndexFileFormatTest
|
||||
protected static void checkReuse(TermsEnum termsEnum, int firstFlags, int secondFlags, boolean shouldReuse) throws IOException {
|
||||
PostingsEnum postings1 = termsEnum.postings(null, firstFlags);
|
||||
PostingsEnum postings2 = termsEnum.postings(postings1, secondFlags);
|
||||
if (shouldReuse)
|
||||
if (shouldReuse) {
|
||||
assertSame("Expected PostingsEnum " + postings1.getClass().getName() + " to be reused", postings1, postings2);
|
||||
else
|
||||
} else {
|
||||
assertNotSame("Expected PostingsEnum " + postings1.getClass().getName() + " to not be reused", postings1, postings2);
|
||||
}
|
||||
}
|
||||
|
||||
public void testJustEmptyField() throws Exception {
|
||||
@ -337,61 +337,104 @@ public abstract class BasePostingsFormatTestCase extends BaseIndexFileFormatTest
|
||||
// TODO: would be better to use / delegate to the current
|
||||
// Codec returned by getCodec()
|
||||
|
||||
iwc.setCodec(new AssertingCodec() {
|
||||
iwc.setCodec(new FilterCodec(getCodec().getName(), getCodec()) {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
public PostingsFormat postingsFormat() {
|
||||
|
||||
PostingsFormat p = getCodec().postingsFormat();
|
||||
if (p instanceof PerFieldPostingsFormat) {
|
||||
p = ((PerFieldPostingsFormat) p).getPostingsFormatForField(field);
|
||||
}
|
||||
final PostingsFormat defaultPostingsFormat = p;
|
||||
final PostingsFormat defaultPostingsFormat = delegate.postingsFormat();
|
||||
|
||||
final Thread mainThread = Thread.currentThread();
|
||||
|
||||
if (field.equals("body")) {
|
||||
// A PF that counts up some stats and then in
|
||||
// the end we verify the stats match what the
|
||||
// final IndexReader says, just to exercise the
|
||||
// new freedom of iterating the postings more
|
||||
// than once at flush/merge:
|
||||
|
||||
// A PF that counts up some stats and then in
|
||||
// the end we verify the stats match what the
|
||||
// final IndexReader says, just to exercise the
|
||||
// new freedom of iterating the postings more
|
||||
// than once at flush/merge:
|
||||
return new PostingsFormat(defaultPostingsFormat.getName()) {
|
||||
|
||||
return new PostingsFormat(defaultPostingsFormat.getName()) {
|
||||
@Override
|
||||
public FieldsConsumer fieldsConsumer(final SegmentWriteState state) throws IOException {
|
||||
|
||||
@Override
|
||||
public FieldsConsumer fieldsConsumer(final SegmentWriteState state) throws IOException {
|
||||
final FieldsConsumer fieldsConsumer = defaultPostingsFormat.fieldsConsumer(state);
|
||||
|
||||
final FieldsConsumer fieldsConsumer = defaultPostingsFormat.fieldsConsumer(state);
|
||||
return new FieldsConsumer() {
|
||||
@Override
|
||||
public void write(Fields fields) throws IOException {
|
||||
fieldsConsumer.write(fields);
|
||||
|
||||
return new FieldsConsumer() {
|
||||
@Override
|
||||
public void write(Fields fields) throws IOException {
|
||||
fieldsConsumer.write(fields);
|
||||
boolean isMerge = state.context.context == IOContext.Context.MERGE;
|
||||
|
||||
boolean isMerge = state.context.context == IOContext.Context.MERGE;
|
||||
// We only use one thread for flushing
|
||||
// in this test:
|
||||
assert isMerge || Thread.currentThread() == mainThread;
|
||||
|
||||
// We only use one thread for flushing
|
||||
// in this test:
|
||||
assert isMerge || Thread.currentThread() == mainThread;
|
||||
// We iterate the provided TermsEnum
|
||||
// twice, so we excercise this new freedom
|
||||
// with the inverted API; if
|
||||
// addOnSecondPass is true, we add up
|
||||
// term stats on the 2nd iteration:
|
||||
boolean addOnSecondPass = random().nextBoolean();
|
||||
|
||||
// We iterate the provided TermsEnum
|
||||
// twice, so we excercise this new freedom
|
||||
// with the inverted API; if
|
||||
// addOnSecondPass is true, we add up
|
||||
// term stats on the 2nd iteration:
|
||||
boolean addOnSecondPass = random().nextBoolean();
|
||||
//System.out.println("write isMerge=" + isMerge + " 2ndPass=" + addOnSecondPass);
|
||||
|
||||
//System.out.println("write isMerge=" + isMerge + " 2ndPass=" + addOnSecondPass);
|
||||
// Gather our own stats:
|
||||
Terms terms = fields.terms("body");
|
||||
assert terms != null;
|
||||
|
||||
// Gather our own stats:
|
||||
Terms terms = fields.terms("body");
|
||||
assert terms != null;
|
||||
TermsEnum termsEnum = terms.iterator();
|
||||
PostingsEnum docs = null;
|
||||
while(termsEnum.next() != null) {
|
||||
BytesRef term = termsEnum.term();
|
||||
// TODO: also sometimes ask for payloads/offsets?
|
||||
boolean noPositions = random().nextBoolean();
|
||||
if (noPositions) {
|
||||
docs = termsEnum.postings(docs, PostingsEnum.FREQS);
|
||||
} else {
|
||||
docs = termsEnum.postings(null, PostingsEnum.POSITIONS);
|
||||
}
|
||||
int docFreq = 0;
|
||||
long totalTermFreq = 0;
|
||||
while (docs.nextDoc() != PostingsEnum.NO_MORE_DOCS) {
|
||||
docFreq++;
|
||||
totalTermFreq += docs.freq();
|
||||
int limit = TestUtil.nextInt(random(), 1, docs.freq());
|
||||
if (!noPositions) {
|
||||
for (int i = 0; i < limit; i++) {
|
||||
docs.nextPosition();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
TermsEnum termsEnum = terms.iterator();
|
||||
PostingsEnum docs = null;
|
||||
while(termsEnum.next() != null) {
|
||||
BytesRef term = termsEnum.term();
|
||||
String termString = term.utf8ToString();
|
||||
|
||||
// During merge we should only see terms
|
||||
// we had already seen during a
|
||||
// previous flush:
|
||||
assertTrue(isMerge==false || termFreqs.containsKey(termString));
|
||||
|
||||
if (isMerge == false) {
|
||||
if (addOnSecondPass == false) {
|
||||
TermFreqs tf = termFreqs.get(termString);
|
||||
if (tf == null) {
|
||||
tf = new TermFreqs();
|
||||
termFreqs.put(termString, tf);
|
||||
}
|
||||
tf.docFreq += docFreq;
|
||||
tf.totalTermFreq += totalTermFreq;
|
||||
sumDocFreq.addAndGet(docFreq);
|
||||
sumTotalTermFreq.addAndGet(totalTermFreq);
|
||||
} else if (termFreqs.containsKey(termString) == false) {
|
||||
// Add placeholder (2nd pass will
|
||||
// set its counts):
|
||||
termFreqs.put(termString, new TermFreqs());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Also test seeking the TermsEnum:
|
||||
for(String term : termFreqs.keySet()) {
|
||||
if (termsEnum.seekExact(new BytesRef(term))) {
|
||||
// TODO: also sometimes ask for payloads/offsets?
|
||||
boolean noPositions = random().nextBoolean();
|
||||
if (noPositions) {
|
||||
@ -399,6 +442,7 @@ public abstract class BasePostingsFormatTestCase extends BaseIndexFileFormatTest
|
||||
} else {
|
||||
docs = termsEnum.postings(null, PostingsEnum.POSITIONS);
|
||||
}
|
||||
|
||||
int docFreq = 0;
|
||||
long totalTermFreq = 0;
|
||||
while (docs.nextDoc() != PostingsEnum.NO_MORE_DOCS) {
|
||||
@ -412,96 +456,43 @@ public abstract class BasePostingsFormatTestCase extends BaseIndexFileFormatTest
|
||||
}
|
||||
}
|
||||
|
||||
String termString = term.utf8ToString();
|
||||
|
||||
// During merge we should only see terms
|
||||
// we had already seen during a
|
||||
// previous flush:
|
||||
assertTrue(isMerge==false || termFreqs.containsKey(termString));
|
||||
|
||||
if (isMerge == false) {
|
||||
if (addOnSecondPass == false) {
|
||||
TermFreqs tf = termFreqs.get(termString);
|
||||
if (tf == null) {
|
||||
tf = new TermFreqs();
|
||||
termFreqs.put(termString, tf);
|
||||
}
|
||||
tf.docFreq += docFreq;
|
||||
tf.totalTermFreq += totalTermFreq;
|
||||
sumDocFreq.addAndGet(docFreq);
|
||||
sumTotalTermFreq.addAndGet(totalTermFreq);
|
||||
} else if (termFreqs.containsKey(termString) == false) {
|
||||
// Add placeholder (2nd pass will
|
||||
// set its counts):
|
||||
termFreqs.put(termString, new TermFreqs());
|
||||
}
|
||||
if (isMerge == false && addOnSecondPass) {
|
||||
TermFreqs tf = termFreqs.get(term);
|
||||
assert tf != null;
|
||||
tf.docFreq += docFreq;
|
||||
tf.totalTermFreq += totalTermFreq;
|
||||
sumDocFreq.addAndGet(docFreq);
|
||||
sumTotalTermFreq.addAndGet(totalTermFreq);
|
||||
}
|
||||
}
|
||||
|
||||
// Also test seeking the TermsEnum:
|
||||
for(String term : termFreqs.keySet()) {
|
||||
if (termsEnum.seekExact(new BytesRef(term))) {
|
||||
// TODO: also sometimes ask for payloads/offsets?
|
||||
boolean noPositions = random().nextBoolean();
|
||||
if (noPositions) {
|
||||
docs = termsEnum.postings(docs, PostingsEnum.FREQS);
|
||||
} else {
|
||||
docs = termsEnum.postings(null, PostingsEnum.POSITIONS);
|
||||
}
|
||||
|
||||
int docFreq = 0;
|
||||
long totalTermFreq = 0;
|
||||
while (docs.nextDoc() != PostingsEnum.NO_MORE_DOCS) {
|
||||
docFreq++;
|
||||
totalTermFreq += docs.freq();
|
||||
int limit = TestUtil.nextInt(random(), 1, docs.freq());
|
||||
if (!noPositions) {
|
||||
for (int i = 0; i < limit; i++) {
|
||||
docs.nextPosition();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (isMerge == false && addOnSecondPass) {
|
||||
TermFreqs tf = termFreqs.get(term);
|
||||
assert tf != null;
|
||||
tf.docFreq += docFreq;
|
||||
tf.totalTermFreq += totalTermFreq;
|
||||
sumDocFreq.addAndGet(docFreq);
|
||||
sumTotalTermFreq.addAndGet(totalTermFreq);
|
||||
}
|
||||
|
||||
//System.out.println(" term=" + term + " docFreq=" + docFreq + " ttDF=" + termToDocFreq.get(term));
|
||||
assertTrue(docFreq <= termFreqs.get(term).docFreq);
|
||||
assertTrue(totalTermFreq <= termFreqs.get(term).totalTermFreq);
|
||||
}
|
||||
}
|
||||
|
||||
// Also test seekCeil
|
||||
for(int iter=0;iter<10;iter++) {
|
||||
BytesRef term = new BytesRef(TestUtil.randomRealisticUnicodeString(random()));
|
||||
SeekStatus status = termsEnum.seekCeil(term);
|
||||
if (status == SeekStatus.NOT_FOUND) {
|
||||
assertTrue(term.compareTo(termsEnum.term()) < 0);
|
||||
}
|
||||
//System.out.println(" term=" + term + " docFreq=" + docFreq + " ttDF=" + termToDocFreq.get(term));
|
||||
assertTrue(docFreq <= termFreqs.get(term).docFreq);
|
||||
assertTrue(totalTermFreq <= termFreqs.get(term).totalTermFreq);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
fieldsConsumer.close();
|
||||
// Also test seekCeil
|
||||
for(int iter=0;iter<10;iter++) {
|
||||
BytesRef term = new BytesRef(TestUtil.randomRealisticUnicodeString(random()));
|
||||
SeekStatus status = termsEnum.seekCeil(term);
|
||||
if (status == SeekStatus.NOT_FOUND) {
|
||||
assertTrue(term.compareTo(termsEnum.term()) < 0);
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
|
||||
return defaultPostingsFormat.fieldsProducer(state);
|
||||
}
|
||||
};
|
||||
} else {
|
||||
return defaultPostingsFormat;
|
||||
}
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
fieldsConsumer.close();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
|
||||
return defaultPostingsFormat.fieldsProducer(state);
|
||||
}
|
||||
};
|
||||
}
|
||||
});
|
||||
|
||||
@ -512,8 +503,10 @@ public abstract class BasePostingsFormatTestCase extends BaseIndexFileFormatTest
|
||||
int bytesIndexed = 0;
|
||||
while (bytesIndexed < bytesToIndex) {
|
||||
Document doc = docs.nextDoc();
|
||||
w.addDocument(doc);
|
||||
bytesIndexed += RamUsageTester.sizeOf(doc);
|
||||
Document justBodyDoc = new Document();
|
||||
justBodyDoc.add(doc.getField("body"));
|
||||
w.addDocument(justBodyDoc);
|
||||
bytesIndexed += RamUsageTester.sizeOf(justBodyDoc);
|
||||
}
|
||||
|
||||
IndexReader r = w.getReader();
|
||||
|
Loading…
x
Reference in New Issue
Block a user