![JAR search and dependency download from the Maven repository](/logo.png)
org.apache.lucene.index.RandomPostingsTester Maven / Gradle / Ivy
Go to download
Show more of this group Show more artifacts with this name
Show all versions of lucene-test-framework Show documentation
Show all versions of lucene-test-framework Show documentation
Apache Lucene (module: test-framework)
/*
* 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.
*/
package org.apache.lucene.index;
import java.io.IOException;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.NoSuchElementException;
import java.util.Random;
import java.util.Set;
import java.util.SortedMap;
import java.util.TreeMap;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FlushInfo;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.UnicodeUtil;
import org.apache.lucene.util.Version;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.AutomatonTestUtil;
import org.apache.lucene.util.automaton.AutomatonTestUtil.RandomAcceptedStrings;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
/** Helper class extracted from BasePostingsFormatTestCase to exercise a postings format. */
public class RandomPostingsTester {
/** Which features to test. */
public enum Option {
// Sometimes use .advance():
SKIPPING,
// Sometimes reuse the PostingsEnum across terms:
REUSE_ENUMS,
// Sometimes pass non-null live docs:
LIVE_DOCS,
// Sometimes seek to term using previously saved TermState:
TERM_STATE,
// Sometimes don't fully consume docs from the enum
PARTIAL_DOC_CONSUME,
// Sometimes don't fully consume positions at each doc
PARTIAL_POS_CONSUME,
// Sometimes check payloads
PAYLOADS,
// Test w/ multiple threads
THREADS
};
private long totalPostings;
private long totalPayloadBytes;
// Holds all postings:
private Map> fields;
private FieldInfos fieldInfos;
List allTerms;
private int maxDoc;
final Random random;
public RandomPostingsTester(Random random) throws IOException {
fields = new TreeMap<>();
this.random = random;
final int numFields = TestUtil.nextInt(random, 1, 5);
if (LuceneTestCase.VERBOSE) {
System.out.println("TEST: " + numFields + " fields");
}
maxDoc = 0;
FieldInfo[] fieldInfoArray = new FieldInfo[numFields];
int fieldUpto = 0;
while (fieldUpto < numFields) {
String field = TestUtil.randomSimpleString(random);
if (fields.containsKey(field)) {
continue;
}
fieldInfoArray[fieldUpto] = new FieldInfo(field, fieldUpto, false, false, true,
IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS,
DocValuesType.NONE, -1, new HashMap<>(),
0, 0);
fieldUpto++;
SortedMap postings = new TreeMap<>();
fields.put(field, postings);
Set seenTerms = new HashSet<>();
int numTerms;
if (random.nextInt(10) == 7) {
numTerms = LuceneTestCase.atLeast(random, 50);
} else {
numTerms = TestUtil.nextInt(random, 2, 20);
}
while (postings.size() < numTerms) {
int termUpto = postings.size();
// Cannot contain surrogates else default Java string sort order (by UTF16 code unit) is different from Lucene:
String term = TestUtil.randomSimpleString(random);
if (seenTerms.contains(term)) {
continue;
}
seenTerms.add(term);
if (LuceneTestCase.TEST_NIGHTLY && termUpto == 0 && fieldUpto == 1) {
// Make 1 big term:
term = "big_" + term;
} else if (termUpto == 1 && fieldUpto == 1) {
// Make 1 medium term:
term = "medium_" + term;
} else if (random.nextBoolean()) {
// Low freq term:
term = "low_" + term;
} else {
// Very low freq term (don't multiply by RANDOM_MULTIPLIER):
term = "verylow_" + term;
}
long termSeed = random.nextLong();
postings.put(new BytesRef(term), new SeedAndOrd(termSeed));
// NOTE: sort of silly: we enum all the docs just to
// get the maxDoc
PostingsEnum postingsEnum = getSeedPostings(term, termSeed, IndexOptions.DOCS, true);
int doc;
int lastDoc = 0;
while((doc = postingsEnum.nextDoc()) != PostingsEnum.NO_MORE_DOCS) {
lastDoc = doc;
}
maxDoc = Math.max(lastDoc, maxDoc);
}
// assign ords
long ord = 0;
for(SeedAndOrd ent : postings.values()) {
ent.ord = ord++;
}
}
fieldInfos = new FieldInfos(fieldInfoArray);
// It's the count, not the last docID:
maxDoc++;
allTerms = new ArrayList<>();
for(Map.Entry> fieldEnt : fields.entrySet()) {
String field = fieldEnt.getKey();
long ord = 0;
for(Map.Entry termEnt : fieldEnt.getValue().entrySet()) {
allTerms.add(new FieldAndTerm(field, termEnt.getKey(), ord++));
}
}
if (LuceneTestCase.VERBOSE) {
System.out.println("TEST: done init postings; " + allTerms.size() + " total terms, across " + fieldInfos.size() + " fields");
}
}
public static SeedPostings getSeedPostings(String term, long seed, IndexOptions options, boolean allowPayloads) {
int minDocFreq, maxDocFreq;
if (term.startsWith("big_")) {
minDocFreq = LuceneTestCase.RANDOM_MULTIPLIER * 50000;
maxDocFreq = LuceneTestCase.RANDOM_MULTIPLIER * 70000;
} else if (term.startsWith("medium_")) {
minDocFreq = LuceneTestCase.RANDOM_MULTIPLIER * 3000;
maxDocFreq = LuceneTestCase.RANDOM_MULTIPLIER * 6000;
} else if (term.startsWith("low_")) {
minDocFreq = LuceneTestCase.RANDOM_MULTIPLIER;
maxDocFreq = LuceneTestCase.RANDOM_MULTIPLIER * 40;
} else {
minDocFreq = 1;
maxDocFreq = 3;
}
return new SeedPostings(seed, minDocFreq, maxDocFreq, options, allowPayloads);
}
/** Given the same random seed this always enumerates the
* same random postings */
public static class SeedPostings extends PostingsEnum {
// Used only to generate docIDs; this way if you pull w/
// or w/o positions you get the same docID sequence:
private final Random docRandom;
private final Random random;
public int docFreq;
private final int maxDocSpacing;
private final int payloadSize;
private final boolean fixedPayloads;
private final BytesRef payload;
private final boolean doPositions;
private final boolean allowPayloads;
private int docID = -1;
private int freq;
public int upto;
private int pos;
private int offset;
private int startOffset;
private int endOffset;
private int posSpacing;
private int posUpto;
public SeedPostings(long seed, int minDocFreq, int maxDocFreq, IndexOptions options, boolean allowPayloads) {
random = new Random(seed);
docRandom = new Random(random.nextLong());
docFreq = TestUtil.nextInt(random, minDocFreq, maxDocFreq);
this.allowPayloads = allowPayloads;
// TODO: more realistic to inversely tie this to numDocs:
maxDocSpacing = TestUtil.nextInt(random, 1, 100);
if (random.nextInt(10) == 7) {
// 10% of the time create big payloads:
payloadSize = 1 + random.nextInt(3);
} else {
payloadSize = 1 + random.nextInt(1);
}
fixedPayloads = random.nextBoolean();
byte[] payloadBytes = new byte[payloadSize];
payload = new BytesRef(payloadBytes);
doPositions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS.compareTo(options) <= 0;
}
@Override
public int nextDoc() {
while(true) {
_nextDoc();
return docID;
}
}
private int _nextDoc() {
if (docID == -1) {
docID = 0;
}
// Must consume random:
while(posUpto < freq) {
nextPosition();
}
if (upto < docFreq) {
if (upto == 0 && docRandom.nextBoolean()) {
// Sometimes index docID = 0
} else if (maxDocSpacing == 1) {
docID++;
} else {
// TODO: sometimes have a biggish gap here!
docID += TestUtil.nextInt(docRandom, 1, maxDocSpacing);
}
if (random.nextInt(200) == 17) {
freq = TestUtil.nextInt(random, 1, 1000);
} else if (random.nextInt(10) == 17) {
freq = TestUtil.nextInt(random, 1, 20);
} else {
freq = TestUtil.nextInt(random, 1, 4);
}
pos = 0;
offset = 0;
posUpto = 0;
posSpacing = TestUtil.nextInt(random, 1, 100);
upto++;
return docID;
} else {
return docID = NO_MORE_DOCS;
}
}
@Override
public int docID() {
return docID;
}
@Override
public int freq() {
return freq;
}
@Override
public int nextPosition() {
if (!doPositions) {
posUpto = freq;
return -1;
}
assert posUpto < freq;
if (posUpto == 0 && random.nextBoolean()) {
// Sometimes index pos = 0
} else if (posSpacing == 1) {
pos++;
} else {
pos += TestUtil.nextInt(random, 1, posSpacing);
}
if (payloadSize != 0) {
if (fixedPayloads) {
payload.length = payloadSize;
random.nextBytes(payload.bytes);
} else {
int thisPayloadSize = random.nextInt(payloadSize);
if (thisPayloadSize != 0) {
payload.length = payloadSize;
random.nextBytes(payload.bytes);
} else {
payload.length = 0;
}
}
} else {
payload.length = 0;
}
if (!allowPayloads) {
payload.length = 0;
}
startOffset = offset + random.nextInt(5);
endOffset = startOffset + random.nextInt(10);
offset = endOffset;
posUpto++;
return pos;
}
@Override
public int startOffset() {
return startOffset;
}
@Override
public int endOffset() {
return endOffset;
}
@Override
public BytesRef getPayload() {
return payload.length == 0 ? null : payload;
}
@Override
public int advance(int target) throws IOException {
return slowAdvance(target);
}
@Override
public long cost() {
return docFreq;
}
}
/** Holds one field, term and ord. */
public static class FieldAndTerm {
final String field;
final BytesRef term;
final long ord;
public FieldAndTerm(String field, BytesRef term, long ord) {
this.field = field;
this.term = BytesRef.deepCopyOf(term);
this.ord = ord;
}
}
private static class SeedAndOrd {
final long seed;
long ord;
public SeedAndOrd(long seed) {
this.seed = seed;
}
}
private static class SeedFields extends Fields {
final Map> fields;
final FieldInfos fieldInfos;
final IndexOptions maxAllowed;
final boolean allowPayloads;
public SeedFields(Map> fields, FieldInfos fieldInfos, IndexOptions maxAllowed, boolean allowPayloads) {
this.fields = fields;
this.fieldInfos = fieldInfos;
this.maxAllowed = maxAllowed;
this.allowPayloads = allowPayloads;
}
@Override
public Iterator iterator() {
return fields.keySet().iterator();
}
@Override
public Terms terms(String field) {
SortedMap terms = fields.get(field);
if (terms == null) {
return null;
} else {
return new SeedTerms(terms, fieldInfos.fieldInfo(field), maxAllowed, allowPayloads);
}
}
@Override
public int size() {
return fields.size();
}
}
private static class SeedTerms extends Terms {
final SortedMap terms;
final FieldInfo fieldInfo;
final IndexOptions maxAllowed;
final boolean allowPayloads;
public SeedTerms(SortedMap terms, FieldInfo fieldInfo, IndexOptions maxAllowed, boolean allowPayloads) {
this.terms = terms;
this.fieldInfo = fieldInfo;
this.maxAllowed = maxAllowed;
this.allowPayloads = allowPayloads;
}
@Override
public TermsEnum iterator() {
SeedTermsEnum termsEnum = new SeedTermsEnum(terms, maxAllowed, allowPayloads);
termsEnum.reset();
return termsEnum;
}
@Override
public long size() {
return terms.size();
}
@Override
public long getSumTotalTermFreq() {
throw new UnsupportedOperationException();
}
@Override
public long getSumDocFreq() {
throw new UnsupportedOperationException();
}
@Override
public int getDocCount() {
throw new UnsupportedOperationException();
}
@Override
public boolean hasFreqs() {
return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
}
@Override
public boolean hasOffsets() {
return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
}
@Override
public boolean hasPositions() {
return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
}
@Override
public boolean hasPayloads() {
return allowPayloads && fieldInfo.hasPayloads();
}
}
private static class SeedTermsEnum extends TermsEnum {
final SortedMap terms;
final IndexOptions maxAllowed;
final boolean allowPayloads;
private Iterator> iterator;
private Map.Entry current;
public SeedTermsEnum(SortedMap terms, IndexOptions maxAllowed, boolean allowPayloads) {
this.terms = terms;
this.maxAllowed = maxAllowed;
this.allowPayloads = allowPayloads;
}
void reset() {
iterator = terms.entrySet().iterator();
}
@Override
public SeekStatus seekCeil(BytesRef text) {
SortedMap tailMap = terms.tailMap(text);
if (tailMap.isEmpty()) {
return SeekStatus.END;
} else {
iterator = tailMap.entrySet().iterator();
current = iterator.next();
if (tailMap.firstKey().equals(text)) {
return SeekStatus.FOUND;
} else {
return SeekStatus.NOT_FOUND;
}
}
}
@Override
public BytesRef next() {
if (iterator.hasNext()) {
current = iterator.next();
return term();
} else {
return null;
}
}
@Override
public void seekExact(long ord) {
throw new UnsupportedOperationException();
}
@Override
public BytesRef term() {
return current.getKey();
}
@Override
public long ord() {
return current.getValue().ord;
}
@Override
public int docFreq() {
throw new UnsupportedOperationException();
}
@Override
public long totalTermFreq() {
throw new UnsupportedOperationException();
}
@Override
public final PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
if (PostingsEnum.featureRequested(flags, PostingsEnum.POSITIONS)) {
if (maxAllowed.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
return null;
}
if (PostingsEnum.featureRequested(flags, PostingsEnum.OFFSETS) && maxAllowed.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) < 0) {
return null;
}
if (PostingsEnum.featureRequested(flags, PostingsEnum.PAYLOADS) && allowPayloads == false) {
return null;
}
}
if (PostingsEnum.featureRequested(flags, PostingsEnum.FREQS) && maxAllowed.compareTo(IndexOptions.DOCS_AND_FREQS) < 0) {
return null;
}
return getSeedPostings(current.getKey().utf8ToString(), current.getValue().seed, maxAllowed, allowPayloads);
}
}
private static class ThreadState {
// Only used with REUSE option:
public PostingsEnum reusePostingsEnum;
}
private FieldInfos currentFieldInfos;
// maxAllowed = the "highest" we can index, but we will still
// randomly index at lower IndexOption
public FieldsProducer buildIndex(Codec codec, Directory dir, IndexOptions maxAllowed, boolean allowPayloads, boolean alwaysTestMax) throws IOException {
SegmentInfo segmentInfo = new SegmentInfo(dir, Version.LATEST, "_0", maxDoc, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
int maxIndexOption = Arrays.asList(IndexOptions.values()).indexOf(maxAllowed);
if (LuceneTestCase.VERBOSE) {
System.out.println("\nTEST: now build index");
}
// TODO use allowPayloads
FieldInfo[] newFieldInfoArray = new FieldInfo[fields.size()];
for(int fieldUpto=0;fieldUpto= 0 && allowPayloads;
newFieldInfoArray[fieldUpto] = new FieldInfo(oldFieldInfo.name,
fieldUpto,
false,
false,
doPayloads,
indexOptions,
DocValuesType.NONE,
-1,
new HashMap<>(),
0, 0);
}
FieldInfos newFieldInfos = new FieldInfos(newFieldInfoArray);
// Estimate that flushed segment size will be 25% of
// what we use in RAM:
long bytes = totalPostings * 8 + totalPayloadBytes;
SegmentWriteState writeState = new SegmentWriteState(null, dir,
segmentInfo, newFieldInfos,
null, new IOContext(new FlushInfo(maxDoc, bytes)));
Fields seedFields = new SeedFields(fields, newFieldInfos, maxAllowed, allowPayloads);
FieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(writeState);
boolean success = false;
try {
consumer.write(seedFields);
success = true;
} finally {
if (success) {
IOUtils.close(consumer);
} else {
IOUtils.closeWhileHandlingException(consumer);
}
}
if (LuceneTestCase.VERBOSE) {
System.out.println("TEST: after indexing: files=");
for(String file : dir.listAll()) {
System.out.println(" " + file + ": " + dir.fileLength(file) + " bytes");
}
}
currentFieldInfos = newFieldInfos;
SegmentReadState readState = new SegmentReadState(dir, segmentInfo, newFieldInfos, IOContext.READ);
return codec.postingsFormat().fieldsProducer(readState);
}
private void verifyEnum(Random random,
ThreadState threadState,
String field,
BytesRef term,
TermsEnum termsEnum,
// Maximum options (docs/freqs/positions/offsets) to test:
IndexOptions maxTestOptions,
IndexOptions maxIndexOptions,
EnumSet
© 2015 - 2025 Weber Informatics LLC | Privacy Policy