Revert "LUCENE-9116: Remove long[] from `PostingsWriterBase#encodeTerm`. (#1149)"

This reverts commit d0b4a166e0.
This commit is contained in:
Adrien Grand 2020-01-09 17:37:54 +01:00
parent 22155bf7a7
commit 239d9a6726
41 changed files with 3239 additions and 92 deletions

View File

@ -88,12 +88,6 @@ API Changes
yield Passages sized a little different due to the fact that the sizing pivot is now the center of the first match and
not its left edge.
* LUCENE-9116: PostingsWriterBase and PostingsReaderBase no longer support
setting a field's metadata via a `long[]`. (Adrien Grand)
* LUCENE-9116: The FST and FSTOrd postings formats have been removed.
(Adrien Grand)
New Features
---------------------
(No changes)

View File

@ -154,7 +154,7 @@ public final class Lucene50PostingsReader extends PostingsReaderBase {
}
@Override
public void decodeTerm(DataInput in, FieldInfo fieldInfo, BlockTermState _termState, boolean absolute)
public void decodeTerm(long[] longs, DataInput in, FieldInfo fieldInfo, BlockTermState _termState, boolean absolute)
throws IOException {
final IntBlockTermState termState = (IntBlockTermState) _termState;
final boolean fieldHasPositions = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
@ -167,11 +167,11 @@ public final class Lucene50PostingsReader extends PostingsReaderBase {
termState.payStartFP = 0;
}
termState.docStartFP += in.readVLong();
termState.docStartFP += longs[0];
if (fieldHasPositions) {
termState.posStartFP += in.readVLong();
termState.posStartFP += longs[1];
if (fieldHasOffsets || fieldHasPayloads) {
termState.payStartFP += in.readVLong();
termState.payStartFP += longs[2];
}
}
if (termState.docFreq == 1) {

View File

@ -187,11 +187,20 @@ public final class Lucene50PostingsWriter extends PushPostingsWriterBase {
}
@Override
public void setField(FieldInfo fieldInfo) {
public int setField(FieldInfo fieldInfo) {
super.setField(fieldInfo);
skipWriter.setField(writePositions, writeOffsets, writePayloads);
lastState = emptyState;
fieldHasNorms = fieldInfo.hasNorms();
if (writePositions) {
if (writePayloads || writeOffsets) {
return 3; // doc + pos + pay FP
} else {
return 2; // doc + pos FP
}
} else {
return 1; // doc FP
}
}
@Override
@ -454,16 +463,16 @@ public final class Lucene50PostingsWriter extends PushPostingsWriterBase {
}
@Override
public void encodeTerm(DataOutput out, FieldInfo fieldInfo, BlockTermState _state, boolean absolute) throws IOException {
public void encodeTerm(long[] longs, DataOutput out, FieldInfo fieldInfo, BlockTermState _state, boolean absolute) throws IOException {
IntBlockTermState state = (IntBlockTermState)_state;
if (absolute) {
lastState = emptyState;
}
out.writeVLong(state.docStartFP - lastState.docStartFP);
longs[0] = state.docStartFP - lastState.docStartFP;
if (writePositions) {
out.writeVLong(state.posStartFP - lastState.posStartFP);
longs[1] = state.posStartFP - lastState.posStartFP;
if (writePayloads || writeOffsets) {
out.writeVLong(state.payStartFP - lastState.payStartFP);
longs[2] = state.payStartFP - lastState.payStartFP;
}
}
if (state.singletonDocID != -1) {

View File

@ -145,6 +145,7 @@ public class BlockTermsReader extends FieldsProducer {
// when frequencies are omitted, sumDocFreq=totalTermFreq and we only write one value
final long sumDocFreq = fieldInfo.getIndexOptions() == IndexOptions.DOCS ? sumTotalTermFreq : in.readVLong();
final int docCount = in.readVInt();
final int longsSize = in.readVInt();
if (docCount < 0 || docCount > state.segmentInfo.maxDoc()) { // #docs with field must be <= #docs
throw new CorruptIndexException("invalid docCount: " + docCount + " maxDoc: " + state.segmentInfo.maxDoc(), in);
}
@ -154,7 +155,7 @@ public class BlockTermsReader extends FieldsProducer {
if (sumTotalTermFreq < sumDocFreq) { // #positions must be >= #postings
throw new CorruptIndexException("invalid sumTotalTermFreq: " + sumTotalTermFreq + " sumDocFreq: " + sumDocFreq, in);
}
FieldReader previous = fields.put(fieldInfo.name, new FieldReader(fieldInfo, numTerms, termsStartPointer, sumTotalTermFreq, sumDocFreq, docCount));
FieldReader previous = fields.put(fieldInfo.name, new FieldReader(fieldInfo, numTerms, termsStartPointer, sumTotalTermFreq, sumDocFreq, docCount, longsSize));
if (previous != null) {
throw new CorruptIndexException("duplicate fields: " + fieldInfo.name, in);
}
@ -222,8 +223,9 @@ public class BlockTermsReader extends FieldsProducer {
final long sumTotalTermFreq;
final long sumDocFreq;
final int docCount;
final int longsSize;
FieldReader(FieldInfo fieldInfo, long numTerms, long termsStartPointer, long sumTotalTermFreq, long sumDocFreq, int docCount) {
FieldReader(FieldInfo fieldInfo, long numTerms, long termsStartPointer, long sumTotalTermFreq, long sumDocFreq, int docCount, int longsSize) {
assert numTerms > 0;
this.fieldInfo = fieldInfo;
this.numTerms = numTerms;
@ -231,6 +233,7 @@ public class BlockTermsReader extends FieldsProducer {
this.sumTotalTermFreq = sumTotalTermFreq;
this.sumDocFreq = sumDocFreq;
this.docCount = docCount;
this.longsSize = longsSize;
}
@Override
@ -323,6 +326,7 @@ public class BlockTermsReader extends FieldsProducer {
private final ByteArrayDataInput freqReader = new ByteArrayDataInput();
private int metaDataUpto;
private long[] longs;
private byte[] bytes;
private ByteArrayDataInput bytesReader;
@ -339,6 +343,7 @@ public class BlockTermsReader extends FieldsProducer {
termSuffixes = new byte[128];
docFreqBytes = new byte[64];
//System.out.println("BTR.enum init this=" + this + " postingsReader=" + postingsReader);
longs = new long[longsSize];
}
// TODO: we may want an alternate mode here which is
@ -821,7 +826,10 @@ public class BlockTermsReader extends FieldsProducer {
//System.out.println(" totTF=" + state.totalTermFreq);
}
// metadata
postingsReader.decodeTerm(bytesReader, fieldInfo, state, absolute);
for (int i = 0; i < longs.length; i++) {
longs[i] = bytesReader.readVLong();
}
postingsReader.decodeTerm(longs, bytesReader, fieldInfo, state, absolute);
metaDataUpto++;
absolute = false;
}

View File

@ -81,8 +81,9 @@ public class BlockTermsWriter extends FieldsConsumer implements Closeable {
public final long sumTotalTermFreq;
public final long sumDocFreq;
public final int docCount;
public final int longsSize;
public FieldMetaData(FieldInfo fieldInfo, long numTerms, long termsStartPointer, long sumTotalTermFreq, long sumDocFreq, int docCount) {
public FieldMetaData(FieldInfo fieldInfo, long numTerms, long termsStartPointer, long sumTotalTermFreq, long sumDocFreq, int docCount, int longsSize) {
assert numTerms > 0;
this.fieldInfo = fieldInfo;
this.termsStartPointer = termsStartPointer;
@ -90,6 +91,7 @@ public class BlockTermsWriter extends FieldsConsumer implements Closeable {
this.sumTotalTermFreq = sumTotalTermFreq;
this.sumDocFreq = sumDocFreq;
this.docCount = docCount;
this.longsSize = longsSize;
}
}
@ -174,6 +176,7 @@ public class BlockTermsWriter extends FieldsConsumer implements Closeable {
}
out.writeVLong(field.sumDocFreq);
out.writeVInt(field.docCount);
out.writeVInt(field.longsSize);
}
writeTrailer(dirStart);
CodecUtil.writeFooter(out);
@ -203,6 +206,7 @@ public class BlockTermsWriter extends FieldsConsumer implements Closeable {
long sumTotalTermFreq;
long sumDocFreq;
int docCount;
int longsSize;
private TermEntry[] pendingTerms;
@ -222,7 +226,7 @@ public class BlockTermsWriter extends FieldsConsumer implements Closeable {
}
termsStartPointer = out.getFilePointer();
this.postingsWriter = postingsWriter;
postingsWriter.setField(fieldInfo);
this.longsSize = postingsWriter.setField(fieldInfo);
}
private final BytesRefBuilder lastPrevTerm = new BytesRefBuilder();
@ -281,7 +285,8 @@ public class BlockTermsWriter extends FieldsConsumer implements Closeable {
termsStartPointer,
fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0 ? sumTotalTermFreq : -1,
sumDocFreq,
docsSeen.cardinality()));
docsSeen.cardinality(),
longsSize));
}
}
@ -302,6 +307,7 @@ public class BlockTermsWriter extends FieldsConsumer implements Closeable {
}
private final ByteBuffersDataOutput bytesWriter = ByteBuffersDataOutput.newResettableInstance();
private final ByteBuffersDataOutput bufferWriter = ByteBuffersDataOutput.newResettableInstance();
private void flushBlock() throws IOException {
//System.out.println("BTW.flushBlock seg=" + segment + " pendingCount=" + pendingCount + " fp=" + out.getFilePointer());
@ -347,10 +353,16 @@ public class BlockTermsWriter extends FieldsConsumer implements Closeable {
bytesWriter.reset();
// 4th pass: write the metadata
long[] longs = new long[longsSize];
boolean absolute = true;
for(int termCount=0;termCount<pendingCount;termCount++) {
final BlockTermState state = pendingTerms[termCount].state;
postingsWriter.encodeTerm(bytesWriter, fieldInfo, state, absolute);
postingsWriter.encodeTerm(longs, bufferWriter, fieldInfo, state, absolute);
for (int i = 0; i < longsSize; i++) {
bytesWriter.writeVLong(longs[i]);
}
bufferWriter.copyTo(bytesWriter);
bufferWriter.reset();
absolute = false;
}
out.writeVInt(Math.toIntExact(bytesWriter.size()));

View File

@ -130,6 +130,7 @@ public final class OrdsBlockTreeTermsReader extends FieldsProducer {
// when frequencies are omitted, sumDocFreq=totalTermFreq and we only write one value
final long sumDocFreq = fieldInfo.getIndexOptions() == IndexOptions.DOCS ? sumTotalTermFreq : in.readVLong();
final int docCount = in.readVInt();
final int longsSize = in.readVInt();
// System.out.println(" longsSize=" + longsSize);
BytesRef minTerm = readBytesRef(in);
@ -146,7 +147,7 @@ public final class OrdsBlockTreeTermsReader extends FieldsProducer {
final long indexStartFP = indexIn.readVLong();
OrdsFieldReader previous = fields.put(fieldInfo.name,
new OrdsFieldReader(this, fieldInfo, numTerms, rootCode, sumTotalTermFreq, sumDocFreq, docCount,
indexStartFP, indexIn, minTerm, maxTerm));
indexStartFP, longsSize, indexIn, minTerm, maxTerm));
if (previous != null) {
throw new CorruptIndexException("duplicate field: " + fieldInfo.name, in);
}

View File

@ -143,11 +143,12 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
public final long sumTotalTermFreq;
public final long sumDocFreq;
public final int docCount;
private final int longsSize;
public final BytesRef minTerm;
public final BytesRef maxTerm;
public FieldMetaData(FieldInfo fieldInfo, Output rootCode, long numTerms, long indexStartFP,
long sumTotalTermFreq, long sumDocFreq, int docCount,
long sumTotalTermFreq, long sumDocFreq, int docCount, int longsSize,
BytesRef minTerm, BytesRef maxTerm) {
assert numTerms > 0;
this.fieldInfo = fieldInfo;
@ -158,6 +159,7 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
this.sumTotalTermFreq = sumTotalTermFreq;
this.sumDocFreq = sumDocFreq;
this.docCount = docCount;
this.longsSize = longsSize;
this.minTerm = minTerm;
this.maxTerm = maxTerm;
}
@ -422,6 +424,7 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
class TermsWriter {
private final FieldInfo fieldInfo;
private final int longsSize;
private long numTerms;
final FixedBitSet docsSeen;
long sumTotalTermFreq;
@ -436,6 +439,8 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
private final BytesRefBuilder lastTerm = new BytesRefBuilder();
private int[] prefixStarts = new int[8];
private final long[] longs;
// Pending stack of terms and blocks. As terms arrive (in sorted order)
// we append to this stack, and once the top of the stack has enough
// terms starting with a common prefix, we write a new block with
@ -628,7 +633,13 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
}
// Write term meta data
postingsWriter.encodeTerm(metaWriter, fieldInfo, state, absolute);
postingsWriter.encodeTerm(longs, bytesWriter, fieldInfo, state, absolute);
for (int pos = 0; pos < longsSize; pos++) {
assert longs[pos] >= 0;
metaWriter.writeVLong(longs[pos]);
}
bytesWriter.copyTo(metaWriter);
bytesWriter.reset();
absolute = false;
}
totalTermCount = end-start;
@ -673,7 +684,13 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
// separate anymore:
// Write term meta data
postingsWriter.encodeTerm(metaWriter, fieldInfo, state, absolute);
postingsWriter.encodeTerm(longs, bytesWriter, fieldInfo, state, absolute);
for (int pos = 0; pos < longsSize; pos++) {
assert longs[pos] >= 0;
metaWriter.writeVLong(longs[pos]);
}
bytesWriter.copyTo(metaWriter);
bytesWriter.reset();
absolute = false;
totalTermCount++;
@ -746,7 +763,8 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
TermsWriter(FieldInfo fieldInfo) {
this.fieldInfo = fieldInfo;
docsSeen = new FixedBitSet(maxDoc);
postingsWriter.setField(fieldInfo);
this.longsSize = postingsWriter.setField(fieldInfo);
this.longs = new long[longsSize];
}
/** Writes one term's worth of postings. */
@ -856,6 +874,7 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
sumTotalTermFreq,
sumDocFreq,
docsSeen.cardinality(),
longsSize,
minTerm, maxTerm));
} else {
assert docsSeen.cardinality() == 0;
@ -865,6 +884,7 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
private final ByteBuffersDataOutput suffixWriter = ByteBuffersDataOutput.newResettableInstance();
private final ByteBuffersDataOutput statsWriter = ByteBuffersDataOutput.newResettableInstance();
private final ByteBuffersDataOutput metaWriter = ByteBuffersDataOutput.newResettableInstance();
private final ByteBuffersDataOutput bytesWriter = ByteBuffersDataOutput.newResettableInstance();
}
private boolean closed;
@ -896,6 +916,7 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
}
out.writeVLong(field.sumDocFreq);
out.writeVInt(field.docCount);
out.writeVInt(field.longsSize);
indexOut.writeVLong(field.indexStartFP);
writeBytesRef(out, field.minTerm);
writeBytesRef(out, field.maxTerm);

View File

@ -46,6 +46,7 @@ final class OrdsFieldReader extends Terms implements Accountable {
final Output rootCode;
final BytesRef minTerm;
final BytesRef maxTerm;
final int longsSize;
final OrdsBlockTreeTermsReader parent;
final FST<Output> index;
@ -53,7 +54,7 @@ final class OrdsFieldReader extends Terms implements Accountable {
OrdsFieldReader(OrdsBlockTreeTermsReader parent, FieldInfo fieldInfo, long numTerms,
Output rootCode, long sumTotalTermFreq, long sumDocFreq, int docCount,
long indexStartFP, IndexInput indexIn, BytesRef minTerm, BytesRef maxTerm) throws IOException {
long indexStartFP, int longsSize, IndexInput indexIn, BytesRef minTerm, BytesRef maxTerm) throws IOException {
assert numTerms > 0;
this.fieldInfo = fieldInfo;
//DEBUG = BlockTreeTermsReader.DEBUG && fieldInfo.name.equals("id");
@ -64,6 +65,7 @@ final class OrdsFieldReader extends Terms implements Accountable {
this.docCount = docCount;
this.indexStartFP = indexStartFP;
this.rootCode = rootCode;
this.longsSize = longsSize;
this.minTerm = minTerm;
this.maxTerm = maxTerm;
// if (DEBUG) {

View File

@ -84,7 +84,9 @@ final class OrdsIntersectTermsEnumFrame {
final BlockTermState termState;
// metadata
// metadata buffer, holding monotonic values
public long[] longs;
// metadata buffer, holding general values
public byte[] bytes;
ByteArrayDataInput bytesReader;
@ -101,6 +103,7 @@ final class OrdsIntersectTermsEnumFrame {
this.ord = ord;
this.termState = ite.fr.parent.postingsReader.newTermState();
this.termState.totalTermFreq = -1;
this.longs = new long[ite.fr.longsSize];
}
void loadNextFloorBlock() throws IOException {
@ -295,8 +298,11 @@ final class OrdsIntersectTermsEnumFrame {
termState.totalTermFreq = termState.docFreq + statsReader.readVLong();
//if (DEBUG) System.out.println(" totTF=" + state.totalTermFreq);
}
// metadata
ite.fr.parent.postingsReader.decodeTerm(bytesReader, ite.fr.fieldInfo, termState, absolute);
// metadata
for (int i = 0; i < ite.fr.longsSize; i++) {
longs[i] = bytesReader.readVLong();
}
ite.fr.parent.postingsReader.decodeTerm(longs, bytesReader, ite.fr.fieldInfo, termState, absolute);
metaDataUpto++;
absolute = false;

View File

@ -97,7 +97,9 @@ final class OrdsSegmentTermsEnumFrame {
final BlockTermState state;
// metadata
// metadata buffer, holding monotonic values
public long[] longs;
// metadata buffer, holding general values
public byte[] bytes;
ByteArrayDataInput bytesReader;
@ -108,6 +110,7 @@ final class OrdsSegmentTermsEnumFrame {
this.ord = ord;
this.state = ste.fr.parent.postingsReader.newTermState();
this.state.totalTermFreq = -1;
this.longs = new long[ste.fr.longsSize];
}
public void setFloorData(ByteArrayDataInput in, BytesRef source) {
@ -504,8 +507,11 @@ final class OrdsSegmentTermsEnumFrame {
}
//if (DEBUG) System.out.println(" longsSize=" + ste.fr.longsSize);
// metadata
ste.fr.parent.postingsReader.decodeTerm(bytesReader, ste.fr.fieldInfo, state, absolute);
// metadata
for (int i = 0; i < ste.fr.longsSize; i++) {
longs[i] = bytesReader.readVLong();
}
ste.fr.parent.postingsReader.decodeTerm(longs, bytesReader, ste.fr.fieldInfo, state, absolute);
metaDataUpto++;
absolute = false;

View File

@ -0,0 +1,78 @@
/*
* 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.codecs.memory;
import java.io.IOException;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsReader;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsWriter;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.IOUtils;
/**
* FSTOrd term dict + Lucene50PBF
*/
public final class FSTOrdPostingsFormat extends PostingsFormat {
public FSTOrdPostingsFormat() {
super("FSTOrd50");
}
@Override
public String toString() {
return getName();
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase postingsWriter = new Lucene84PostingsWriter(state);
boolean success = false;
try {
FieldsConsumer ret = new FSTOrdTermsWriter(state, postingsWriter);
success = true;
return ret;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(postingsWriter);
}
}
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postingsReader = new Lucene84PostingsReader(state);
boolean success = false;
try {
FieldsProducer ret = new FSTOrdTermsReader(state, postingsReader);
success = true;
return ret;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(postingsReader);
}
}
}
}

View File

@ -0,0 +1,884 @@
/*
* 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.codecs.memory;
import java.io.IOException;
import java.util.ArrayList;
import java.util.BitSet;
import java.util.Collection;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.TreeMap;
import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.ImpactsEnum;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.TermState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.automaton.ByteRunAutomaton;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.fst.BytesRefFSTEnum;
import org.apache.lucene.util.fst.BytesRefFSTEnum.InputOutput;
import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.Outputs;
import org.apache.lucene.util.fst.PositiveIntOutputs;
import org.apache.lucene.util.fst.Util;
/**
* FST-based terms dictionary reader.
*
* The FST index maps each term and its ord, and during seek
* the ord is used to fetch metadata from a single block.
* The term dictionary is fully memory resident.
*
* @lucene.experimental
*/
public class FSTOrdTermsReader extends FieldsProducer {
static final int INTERVAL = FSTOrdTermsWriter.SKIP_INTERVAL;
final TreeMap<String, TermsReader> fields = new TreeMap<>();
final PostingsReaderBase postingsReader;
//static final boolean TEST = false;
public FSTOrdTermsReader(SegmentReadState state, PostingsReaderBase postingsReader) throws IOException {
final String termsIndexFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, FSTOrdTermsWriter.TERMS_INDEX_EXTENSION);
final String termsBlockFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, FSTOrdTermsWriter.TERMS_BLOCK_EXTENSION);
this.postingsReader = postingsReader;
ChecksumIndexInput indexIn = null;
IndexInput blockIn = null;
boolean success = false;
try {
indexIn = state.directory.openChecksumInput(termsIndexFileName, state.context);
blockIn = state.directory.openInput(termsBlockFileName, state.context);
int version = CodecUtil.checkIndexHeader(indexIn, FSTOrdTermsWriter.TERMS_INDEX_CODEC_NAME,
FSTOrdTermsWriter.VERSION_START,
FSTOrdTermsWriter.VERSION_CURRENT,
state.segmentInfo.getId(), state.segmentSuffix);
int version2 = CodecUtil.checkIndexHeader(blockIn, FSTOrdTermsWriter.TERMS_CODEC_NAME,
FSTOrdTermsWriter.VERSION_START,
FSTOrdTermsWriter.VERSION_CURRENT,
state.segmentInfo.getId(), state.segmentSuffix);
if (version != version2) {
throw new CorruptIndexException("Format versions mismatch: index=" + version + ", terms=" + version2, blockIn);
}
CodecUtil.checksumEntireFile(blockIn);
this.postingsReader.init(blockIn, state);
seekDir(blockIn);
final FieldInfos fieldInfos = state.fieldInfos;
final int numFields = blockIn.readVInt();
for (int i = 0; i < numFields; i++) {
FieldInfo fieldInfo = fieldInfos.fieldInfo(blockIn.readVInt());
boolean hasFreq = fieldInfo.getIndexOptions() != IndexOptions.DOCS;
long numTerms = blockIn.readVLong();
long sumTotalTermFreq = blockIn.readVLong();
// if freqs are omitted, sumDocFreq=sumTotalTermFreq and we only write one value
long sumDocFreq = hasFreq ? blockIn.readVLong() : sumTotalTermFreq;
int docCount = blockIn.readVInt();
int longsSize = blockIn.readVInt();
FST<Long> index = new FST<>(indexIn, PositiveIntOutputs.getSingleton());
TermsReader current = new TermsReader(fieldInfo, blockIn, numTerms, sumTotalTermFreq, sumDocFreq, docCount, longsSize, index);
TermsReader previous = fields.put(fieldInfo.name, current);
checkFieldSummary(state.segmentInfo, indexIn, blockIn, current, previous);
}
CodecUtil.checkFooter(indexIn);
success = true;
} finally {
if (success) {
IOUtils.close(indexIn, blockIn);
} else {
IOUtils.closeWhileHandlingException(indexIn, blockIn);
}
}
}
private void seekDir(IndexInput in) throws IOException {
in.seek(in.length() - CodecUtil.footerLength() - 8);
in.seek(in.readLong());
}
private void checkFieldSummary(SegmentInfo info, IndexInput indexIn, IndexInput blockIn, TermsReader field, TermsReader previous) throws IOException {
// #docs with field must be <= #docs
if (field.docCount < 0 || field.docCount > info.maxDoc()) {
throw new CorruptIndexException("invalid docCount: " + field.docCount + " maxDoc: " + info.maxDoc() + " (blockIn=" + blockIn + ")", indexIn);
}
// #postings must be >= #docs with field
if (field.sumDocFreq < field.docCount) {
throw new CorruptIndexException("invalid sumDocFreq: " + field.sumDocFreq + " docCount: " + field.docCount + " (blockIn=" + blockIn + ")", indexIn);
}
// #positions must be >= #postings
if (field.sumTotalTermFreq < field.sumDocFreq) {
throw new CorruptIndexException("invalid sumTotalTermFreq: " + field.sumTotalTermFreq + " sumDocFreq: " + field.sumDocFreq + " (blockIn=" + blockIn + ")", indexIn);
}
if (previous != null) {
throw new CorruptIndexException("duplicate fields: " + field.fieldInfo.name + " (blockIn=" + blockIn + ")", indexIn);
}
}
@Override
public Iterator<String> iterator() {
return Collections.unmodifiableSet(fields.keySet()).iterator();
}
@Override
public Terms terms(String field) throws IOException {
assert field != null;
return fields.get(field);
}
@Override
public int size() {
return fields.size();
}
@Override
public void close() throws IOException {
try {
IOUtils.close(postingsReader);
} finally {
fields.clear();
}
}
final class TermsReader extends Terms implements Accountable {
final FieldInfo fieldInfo;
final long numTerms;
final long sumTotalTermFreq;
final long sumDocFreq;
final int docCount;
final int longsSize;
final FST<Long> index;
final int numSkipInfo;
final long[] skipInfo;
final byte[] statsBlock;
final byte[] metaLongsBlock;
final byte[] metaBytesBlock;
TermsReader(FieldInfo fieldInfo, IndexInput blockIn, long numTerms, long sumTotalTermFreq, long sumDocFreq, int docCount, int longsSize, FST<Long> index) throws IOException {
this.fieldInfo = fieldInfo;
this.numTerms = numTerms;
this.sumTotalTermFreq = sumTotalTermFreq;
this.sumDocFreq = sumDocFreq;
this.docCount = docCount;
this.longsSize = longsSize;
this.index = index;
assert (numTerms & (~0xffffffffL)) == 0;
final int numBlocks = (int)(numTerms + INTERVAL - 1) / INTERVAL;
this.numSkipInfo = longsSize + 3;
this.skipInfo = new long[numBlocks * numSkipInfo];
this.statsBlock = new byte[(int)blockIn.readVLong()];
this.metaLongsBlock = new byte[(int)blockIn.readVLong()];
this.metaBytesBlock = new byte[(int)blockIn.readVLong()];
int last = 0, next = 0;
for (int i = 1; i < numBlocks; i++) {
next = numSkipInfo * i;
for (int j = 0; j < numSkipInfo; j++) {
skipInfo[next + j] = skipInfo[last + j] + blockIn.readVLong();
}
last = next;
}
blockIn.readBytes(statsBlock, 0, statsBlock.length);
blockIn.readBytes(metaLongsBlock, 0, metaLongsBlock.length);
blockIn.readBytes(metaBytesBlock, 0, metaBytesBlock.length);
}
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 fieldInfo.hasPayloads();
}
@Override
public long size() {
return numTerms;
}
@Override
public long getSumTotalTermFreq() {
return sumTotalTermFreq;
}
@Override
public long getSumDocFreq() throws IOException {
return sumDocFreq;
}
@Override
public int getDocCount() throws IOException {
return docCount;
}
@Override
public TermsEnum iterator() throws IOException {
return new SegmentTermsEnum();
}
@Override
public TermsEnum intersect(CompiledAutomaton compiled, BytesRef startTerm) throws IOException {
if (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
}
return new IntersectTermsEnum(compiled, startTerm);
}
@Override
public long ramBytesUsed() {
long ramBytesUsed = 0;
if (index != null) {
ramBytesUsed += index.ramBytesUsed();
ramBytesUsed += RamUsageEstimator.sizeOf(metaBytesBlock);
ramBytesUsed += RamUsageEstimator.sizeOf(metaLongsBlock);
ramBytesUsed += RamUsageEstimator.sizeOf(skipInfo);
ramBytesUsed += RamUsageEstimator.sizeOf(statsBlock);
}
return ramBytesUsed;
}
@Override
public Collection<Accountable> getChildResources() {
if (index == null) {
return Collections.emptyList();
} else {
return Collections.singletonList(Accountables.namedAccountable("terms", index));
}
}
@Override
public String toString() {
return "FSTOrdTerms(terms=" + numTerms + ",postings=" + sumDocFreq + ",positions=" + sumTotalTermFreq + ",docs=" + docCount + ")";
}
// Only wraps common operations for PBF interact
abstract class BaseTermsEnum extends org.apache.lucene.index.BaseTermsEnum {
/* Current term's ord, starts from 0 */
long ord;
/* Current term stats + decoded metadata (customized by PBF) */
final BlockTermState state;
/* Datainput to load stats & metadata */
final ByteArrayDataInput statsReader = new ByteArrayDataInput();
final ByteArrayDataInput metaLongsReader = new ByteArrayDataInput();
final ByteArrayDataInput metaBytesReader = new ByteArrayDataInput();
/* To which block is buffered */
int statsBlockOrd;
int metaBlockOrd;
/* Current buffered metadata (long[] & byte[]) */
long[][] longs;
int[] bytesStart;
int[] bytesLength;
/* Current buffered stats (df & ttf) */
int[] docFreq;
long[] totalTermFreq;
BaseTermsEnum() throws IOException {
this.state = postingsReader.newTermState();
this.statsReader.reset(statsBlock);
this.metaLongsReader.reset(metaLongsBlock);
this.metaBytesReader.reset(metaBytesBlock);
this.longs = new long[INTERVAL][longsSize];
this.bytesStart = new int[INTERVAL];
this.bytesLength = new int[INTERVAL];
this.docFreq = new int[INTERVAL];
this.totalTermFreq = new long[INTERVAL];
this.statsBlockOrd = -1;
this.metaBlockOrd = -1;
}
/** Decodes stats data into term state */
void decodeStats() throws IOException {
final int upto = (int)ord % INTERVAL;
final int oldBlockOrd = statsBlockOrd;
statsBlockOrd = (int)ord / INTERVAL;
if (oldBlockOrd != statsBlockOrd) {
refillStats();
}
state.docFreq = docFreq[upto];
state.totalTermFreq = totalTermFreq[upto];
}
/** Let PBF decode metadata */
void decodeMetaData() throws IOException {
final int upto = (int)ord % INTERVAL;
final int oldBlockOrd = metaBlockOrd;
metaBlockOrd = (int)ord / INTERVAL;
if (metaBlockOrd != oldBlockOrd) {
refillMetadata();
}
metaBytesReader.setPosition(bytesStart[upto]);
postingsReader.decodeTerm(longs[upto], metaBytesReader, fieldInfo, state, true);
}
/** Load current stats shard */
final void refillStats() throws IOException {
final int offset = statsBlockOrd * numSkipInfo;
final int statsFP = (int)skipInfo[offset];
statsReader.setPosition(statsFP);
for (int i = 0; i < INTERVAL && !statsReader.eof(); i++) {
int code = statsReader.readVInt();
if (hasFreqs()) {
docFreq[i] = (code >>> 1);
if ((code & 1) == 1) {
totalTermFreq[i] = docFreq[i];
} else {
totalTermFreq[i] = docFreq[i] + statsReader.readVLong();
}
} else {
docFreq[i] = code;
totalTermFreq[i] = code;
}
}
}
/** Load current metadata shard */
final void refillMetadata() throws IOException {
final int offset = metaBlockOrd * numSkipInfo;
final int metaLongsFP = (int)skipInfo[offset + 1];
final int metaBytesFP = (int)skipInfo[offset + 2];
metaLongsReader.setPosition(metaLongsFP);
for (int j = 0; j < longsSize; j++) {
longs[0][j] = skipInfo[offset + 3 + j] + metaLongsReader.readVLong();
}
bytesStart[0] = metaBytesFP;
bytesLength[0] = (int)metaLongsReader.readVLong();
for (int i = 1; i < INTERVAL && !metaLongsReader.eof(); i++) {
for (int j = 0; j < longsSize; j++) {
longs[i][j] = longs[i-1][j] + metaLongsReader.readVLong();
}
bytesStart[i] = bytesStart[i-1] + bytesLength[i-1];
bytesLength[i] = (int)metaLongsReader.readVLong();
}
}
@Override
public TermState termState() throws IOException {
decodeMetaData();
return state.clone();
}
@Override
public int docFreq() throws IOException {
return state.docFreq;
}
@Override
public long totalTermFreq() throws IOException {
return state.totalTermFreq;
}
@Override
public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
decodeMetaData();
return postingsReader.postings(fieldInfo, state, reuse, flags);
}
@Override
public ImpactsEnum impacts(int flags) throws IOException {
decodeMetaData();
return postingsReader.impacts(fieldInfo, state, flags);
}
// TODO: this can be achieved by making use of Util.getByOutput()
// and should have related tests
@Override
public void seekExact(long ord) throws IOException {
throw new UnsupportedOperationException();
}
@Override
public long ord() {
throw new UnsupportedOperationException();
}
}
// Iterates through all terms in this field
private final class SegmentTermsEnum extends BaseTermsEnum {
final BytesRefFSTEnum<Long> fstEnum;
/* Current term, null when enum ends or unpositioned */
BytesRef term;
/* True when current term's metadata is decoded */
boolean decoded;
/* True when current enum is 'positioned' by seekExact(TermState) */
boolean seekPending;
SegmentTermsEnum() throws IOException {
this.fstEnum = new BytesRefFSTEnum<>(index);
this.decoded = false;
this.seekPending = false;
}
@Override
public BytesRef term() throws IOException {
return term;
}
@Override
void decodeMetaData() throws IOException {
if (!decoded && !seekPending) {
super.decodeMetaData();
decoded = true;
}
}
// Update current enum according to FSTEnum
void updateEnum(final InputOutput<Long> pair) throws IOException {
if (pair == null) {
term = null;
} else {
term = pair.input;
ord = pair.output;
decodeStats();
}
decoded = false;
seekPending = false;
}
@Override
public BytesRef next() throws IOException {
if (seekPending) { // previously positioned, but termOutputs not fetched
seekPending = false;
SeekStatus status = seekCeil(term);
assert status == SeekStatus.FOUND; // must positioned on valid term
}
updateEnum(fstEnum.next());
return term;
}
@Override
public boolean seekExact(BytesRef target) throws IOException {
updateEnum(fstEnum.seekExact(target));
return term != null;
}
@Override
public SeekStatus seekCeil(BytesRef target) throws IOException {
updateEnum(fstEnum.seekCeil(target));
if (term == null) {
return SeekStatus.END;
} else {
return term.equals(target) ? SeekStatus.FOUND : SeekStatus.NOT_FOUND;
}
}
@Override
public void seekExact(BytesRef target, TermState otherState) {
if (!target.equals(term)) {
state.copyFrom(otherState);
term = BytesRef.deepCopyOf(target);
seekPending = true;
}
}
}
// Iterates intersect result with automaton (cannot seek!)
private final class IntersectTermsEnum extends BaseTermsEnum {
/* Current term, null when enum ends or unpositioned */
BytesRefBuilder term;
/* True when current term's metadata is decoded */
boolean decoded;
/* True when there is pending term when calling next() */
boolean pending;
/* stack to record how current term is constructed,
* used to accumulate metadata or rewind term:
* level == term.length + 1,
* == 0 when term is null */
Frame[] stack;
int level;
/* term dict fst */
final FST<Long> fst;
final FST.BytesReader fstReader;
final Outputs<Long> fstOutputs;
/* query automaton to intersect with */
final ByteRunAutomaton fsa;
private final class Frame {
/* fst stats */
FST.Arc<Long> arc;
Long output;
/* automaton stats */
int state;
Frame() {
this.arc = new FST.Arc<>();
this.state = -1;
}
public String toString() {
return "arc=" + arc + " state=" + state;
}
}
IntersectTermsEnum(CompiledAutomaton compiled, BytesRef startTerm) throws IOException {
//if (TEST) System.out.println("Enum init, startTerm=" + startTerm);
this.fst = index;
this.fstReader = fst.getBytesReader();
this.fstOutputs = index.outputs;
this.fsa = compiled.runAutomaton;
this.level = -1;
this.stack = new Frame[16];
for (int i = 0 ; i < stack.length; i++) {
this.stack[i] = new Frame();
}
Frame frame;
frame = loadVirtualFrame(newFrame());
this.level++;
frame = loadFirstFrame(newFrame());
pushFrame(frame);
this.decoded = false;
this.pending = false;
if (startTerm == null) {
pending = isAccept(topFrame());
} else {
doSeekCeil(startTerm);
pending = (term == null || !startTerm.equals(term.get())) && isValid(topFrame()) && isAccept(topFrame());
}
}
@Override
public BytesRef term() throws IOException {
return term == null ? null : term.get();
}
@Override
void decodeMetaData() throws IOException {
if (!decoded) {
super.decodeMetaData();
decoded = true;
}
}
@Override
void decodeStats() throws IOException {
ord = topFrame().output;
super.decodeStats();
}
@Override
public SeekStatus seekCeil(BytesRef target) throws IOException {
throw new UnsupportedOperationException();
}
@Override
public BytesRef next() throws IOException {
//if (TEST) System.out.println("Enum next()");
if (pending) {
pending = false;
decodeStats();
return term();
}
decoded = false;
DFS:
while (level > 0) {
Frame frame = newFrame();
if (loadExpandFrame(topFrame(), frame) != null) { // has valid target
pushFrame(frame);
if (isAccept(frame)) { // gotcha
break;
}
continue; // check next target
}
frame = popFrame();
while(level > 0) {
if (loadNextFrame(topFrame(), frame) != null) { // has valid sibling
pushFrame(frame);
if (isAccept(frame)) { // gotcha
break DFS;
}
continue DFS; // check next target
}
frame = popFrame();
}
return null;
}
decodeStats();
return term();
}
BytesRef doSeekCeil(BytesRef target) throws IOException {
//if (TEST) System.out.println("Enum doSeekCeil()");
Frame frame= null;
int label, upto = 0, limit = target.length;
while (upto < limit) { // to target prefix, or ceil label (rewind prefix)
frame = newFrame();
label = target.bytes[upto] & 0xff;
frame = loadCeilFrame(label, topFrame(), frame);
if (frame == null || frame.arc.label() != label) {
break;
}
assert isValid(frame); // target must be fetched from automaton
pushFrame(frame);
upto++;
}
if (upto == limit) { // got target
return term();
}
if (frame != null) { // got larger term('s prefix)
pushFrame(frame);
return isAccept(frame) ? term() : next();
}
while (level > 0) { // got target's prefix, advance to larger term
frame = popFrame();
while (level > 0 && !canRewind(frame)) {
frame = popFrame();
}
if (loadNextFrame(topFrame(), frame) != null) {
pushFrame(frame);
return isAccept(frame) ? term() : next();
}
}
return null;
}
/** Virtual frame, never pop */
Frame loadVirtualFrame(Frame frame) {
frame.output = fstOutputs.getNoOutput();
frame.state = -1;
return frame;
}
/** Load frame for start arc(node) on fst */
Frame loadFirstFrame(Frame frame) {
frame.arc = fst.getFirstArc(frame.arc);
frame.output = frame.arc.output();
frame.state = 0;
return frame;
}
/** Load frame for target arc(node) on fst */
Frame loadExpandFrame(Frame top, Frame frame) throws IOException {
if (!canGrow(top)) {
return null;
}
frame.arc = fst.readFirstRealTargetArc(top.arc.target(), frame.arc, fstReader);
frame.state = fsa.step(top.state, frame.arc.label());
frame.output = frame.arc.output();
//if (TEST) System.out.println(" loadExpand frame="+frame);
if (frame.state == -1) {
return loadNextFrame(top, frame);
}
return frame;
}
/** Load frame for sibling arc(node) on fst */
Frame loadNextFrame(Frame top, Frame frame) throws IOException {
if (!canRewind(frame)) {
return null;
}
while (!frame.arc.isLast()) {
frame.arc = fst.readNextRealArc(frame.arc, fstReader);
frame.output = frame.arc.output();
frame.state = fsa.step(top.state, frame.arc.label());
if (frame.state != -1) {
break;
}
}
//if (TEST) System.out.println(" loadNext frame="+frame);
if (frame.state == -1) {
return null;
}
return frame;
}
/** Load frame for target arc(node) on fst, so that
* arc.label &gt;= label and !fsa.reject(arc.label) */
Frame loadCeilFrame(int label, Frame top, Frame frame) throws IOException {
FST.Arc<Long> arc = frame.arc;
arc = Util.readCeilArc(label, fst, top.arc, arc, fstReader);
if (arc == null) {
return null;
}
frame.state = fsa.step(top.state, arc.label());
//if (TEST) System.out.println(" loadCeil frame="+frame);
if (frame.state == -1) {
return loadNextFrame(top, frame);
}
frame.output = arc.output();
return frame;
}
boolean isAccept(Frame frame) { // reach a term both fst&fsa accepts
return fsa.isAccept(frame.state) && frame.arc.isFinal();
}
boolean isValid(Frame frame) { // reach a prefix both fst&fsa won't reject
return /*frame != null &&*/ frame.state != -1;
}
boolean canGrow(Frame frame) { // can walk forward on both fst&fsa
return frame.state != -1 && FST.targetHasArcs(frame.arc);
}
boolean canRewind(Frame frame) { // can jump to sibling
return !frame.arc.isLast();
}
void pushFrame(Frame frame) {
final FST.Arc<Long> arc = frame.arc;
frame.output = fstOutputs.add(topFrame().output, frame.output);
term = grow(arc.label());
level++;
assert frame == stack[level];
}
Frame popFrame() {
term = shrink();
return stack[level--];
}
Frame newFrame() {
if (level+1 == stack.length) {
final Frame[] temp = new Frame[ArrayUtil.oversize(level+2, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
System.arraycopy(stack, 0, temp, 0, stack.length);
for (int i = stack.length; i < temp.length; i++) {
temp[i] = new Frame();
}
stack = temp;
}
return stack[level+1];
}
Frame topFrame() {
return stack[level];
}
BytesRefBuilder grow(int label) {
if (term == null) {
term = new BytesRefBuilder();
} else {
term.append((byte) label);
}
return term;
}
BytesRefBuilder shrink() {
if (term.length() == 0) {
term = null;
} else {
term.setLength(term.length() - 1);
}
return term;
}
}
}
static<T> void walk(FST<T> fst) throws IOException {
final ArrayList<FST.Arc<T>> queue = new ArrayList<>();
final BitSet seen = new BitSet();
final FST.BytesReader reader = fst.getBytesReader();
final FST.Arc<T> startArc = fst.getFirstArc(new FST.Arc<T>());
queue.add(startArc);
while (!queue.isEmpty()) {
final FST.Arc<T> arc = queue.remove(0);
final long node = arc.target();
//System.out.println(arc);
if (FST.targetHasArcs(arc) && !seen.get((int) node)) {
seen.set((int) node);
fst.readFirstRealTargetArc(node, arc, reader);
while (true) {
queue.add(new FST.Arc<T>().copyFrom(arc));
if (arc.isLast()) {
break;
} else {
fst.readNextRealArc(arc, reader);
}
}
}
}
}
@Override
public long ramBytesUsed() {
long ramBytesUsed = postingsReader.ramBytesUsed();
for (TermsReader r : fields.values()) {
ramBytesUsed += r.ramBytesUsed();
}
return ramBytesUsed;
}
@Override
public Collection<Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>(Accountables.namedAccountables("field", fields));
resources.add(Accountables.namedAccountable("delegate", postingsReader));
return Collections.unmodifiableList(resources);
}
@Override
public String toString() {
return getClass().getSimpleName() + "(fields=" + fields.size() + ",delegate=" + postingsReader + ")";
}
@Override
public void checkIntegrity() throws IOException {
postingsReader.checkIntegrity();
}
}

View File

@ -0,0 +1,386 @@
/*
* 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.codecs.memory;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.IntsRefBuilder;
import org.apache.lucene.util.fst.FSTCompiler;
import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.PositiveIntOutputs;
import org.apache.lucene.util.fst.Util;
/**
* FST-based term dict, using ord as FST output.
*
* The FST holds the mapping between &lt;term, ord&gt;, and
* term's metadata is delta encoded into a single byte block.
*
* Typically the byte block consists of four parts:
* 1. term statistics: docFreq, totalTermFreq;
* 2. monotonic long[], e.g. the pointer to the postings list for that term;
* 3. generic byte[], e.g. other information customized by postings base.
* 4. single-level skip list to speed up metadata decoding by ord.
*
* <p>
* Files:
* <ul>
* <li><tt>.tix</tt>: <a href="#Termindex">Term Index</a></li>
* <li><tt>.tbk</tt>: <a href="#Termblock">Term Block</a></li>
* </ul>
*
* <a name="Termindex"></a>
* <h3>Term Index</h3>
* <p>
* The .tix contains a list of FSTs, one for each field.
* The FST maps a term to its corresponding order in current field.
* </p>
*
* <ul>
* <li>TermIndex(.tix) --&gt; Header, TermFST<sup>NumFields</sup>, Footer</li>
* <li>TermFST --&gt; {@link FST FST&lt;long&gt;}</li>
* <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
* <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
* </ul>
*
* <p>Notes:</p>
* <ul>
* <li>
* Since terms are already sorted before writing to <a href="#Termblock">Term Block</a>,
* their ords can directly used to seek term metadata from term block.
* </li>
* </ul>
*
* <a name="Termblock"></a>
* <h3>Term Block</h3>
* <p>
* The .tbk contains all the statistics and metadata for terms, along with field summary (e.g.
* per-field data like number of documents in current field). For each field, there are four blocks:
* <ul>
* <li>statistics bytes block: contains term statistics; </li>
* <li>metadata longs block: delta-encodes monotonic part of metadata; </li>
* <li>metadata bytes block: encodes other parts of metadata; </li>
* <li>skip block: contains skip data, to speed up metadata seeking and decoding</li>
* </ul>
*
* <p>File Format:</p>
* <ul>
* <li>TermBlock(.tbk) --&gt; Header, <i>PostingsHeader</i>, FieldSummary, DirOffset</li>
* <li>FieldSummary --&gt; NumFields, &lt;FieldNumber, NumTerms, SumTotalTermFreq?, SumDocFreq,
* DocCount, LongsSize, DataBlock &gt; <sup>NumFields</sup>, Footer</li>
*
* <li>DataBlock --&gt; StatsBlockLength, MetaLongsBlockLength, MetaBytesBlockLength,
* SkipBlock, StatsBlock, MetaLongsBlock, MetaBytesBlock </li>
* <li>SkipBlock --&gt; &lt; StatsFPDelta, MetaLongsSkipFPDelta, MetaBytesSkipFPDelta,
* MetaLongsSkipDelta<sup>LongsSize</sup> &gt;<sup>NumTerms</sup>
* <li>StatsBlock --&gt; &lt; DocFreq[Same?], (TotalTermFreq-DocFreq) ? &gt; <sup>NumTerms</sup>
* <li>MetaLongsBlock --&gt; &lt; LongDelta<sup>LongsSize</sup>, BytesSize &gt; <sup>NumTerms</sup>
* <li>MetaBytesBlock --&gt; Byte <sup>MetaBytesBlockLength</sup>
* <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
* <li>DirOffset --&gt; {@link DataOutput#writeLong Uint64}</li>
* <li>NumFields, FieldNumber, DocCount, DocFreq, LongsSize,
* FieldNumber, DocCount --&gt; {@link DataOutput#writeVInt VInt}</li>
* <li>NumTerms, SumTotalTermFreq, SumDocFreq, StatsBlockLength, MetaLongsBlockLength, MetaBytesBlockLength,
* StatsFPDelta, MetaLongsSkipFPDelta, MetaBytesSkipFPDelta, MetaLongsSkipStart, TotalTermFreq,
* LongDelta,--&gt; {@link DataOutput#writeVLong VLong}</li>
* <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
* </ul>
* <p>Notes: </p>
* <ul>
* <li>
* The format of PostingsHeader and MetaBytes are customized by the specific postings implementation:
* they contain arbitrary per-file data (such as parameters or versioning information), and per-term data
* (non-monotonic ones like pulsed postings data).
* </li>
* <li>
* During initialization the reader will load all the blocks into memory. SkipBlock will be decoded, so that during seek
* term dict can lookup file pointers directly. StatsFPDelta, MetaLongsSkipFPDelta, etc. are file offset
* for every SkipInterval's term. MetaLongsSkipDelta is the difference from previous one, which indicates
* the value of preceding metadata longs for every SkipInterval's term.
* </li>
* <li>
* DocFreq is the count of documents which contain the term. TotalTermFreq is the total number of occurrences of the term.
* Usually these two values are the same for long tail terms, therefore one bit is stole from DocFreq to check this case,
* so that encoding of TotalTermFreq may be omitted.
* </li>
* </ul>
*
* @lucene.experimental
*/
public class FSTOrdTermsWriter extends FieldsConsumer {
static final String TERMS_INDEX_EXTENSION = "tix";
static final String TERMS_BLOCK_EXTENSION = "tbk";
static final String TERMS_CODEC_NAME = "FSTOrdTerms";
static final String TERMS_INDEX_CODEC_NAME = "FSTOrdIndex";
public static final int VERSION_START = 2;
public static final int VERSION_CURRENT = VERSION_START;
public static final int SKIP_INTERVAL = 8;
final PostingsWriterBase postingsWriter;
final FieldInfos fieldInfos;
final int maxDoc;
final List<FieldMetaData> fields = new ArrayList<>();
IndexOutput blockOut = null;
IndexOutput indexOut = null;
public FSTOrdTermsWriter(SegmentWriteState state, PostingsWriterBase postingsWriter) throws IOException {
final String termsIndexFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_INDEX_EXTENSION);
final String termsBlockFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_BLOCK_EXTENSION);
this.postingsWriter = postingsWriter;
this.fieldInfos = state.fieldInfos;
this.maxDoc = state.segmentInfo.maxDoc();
boolean success = false;
try {
this.indexOut = state.directory.createOutput(termsIndexFileName, state.context);
this.blockOut = state.directory.createOutput(termsBlockFileName, state.context);
CodecUtil.writeIndexHeader(indexOut, TERMS_INDEX_CODEC_NAME, VERSION_CURRENT,
state.segmentInfo.getId(), state.segmentSuffix);
CodecUtil.writeIndexHeader(blockOut, TERMS_CODEC_NAME, VERSION_CURRENT,
state.segmentInfo.getId(), state.segmentSuffix);
this.postingsWriter.init(blockOut, state);
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(indexOut, blockOut);
}
}
}
@Override
public void write(Fields fields, NormsProducer norms) throws IOException {
for(String field : fields) {
Terms terms = fields.terms(field);
if (terms == null) {
continue;
}
FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
boolean hasFreq = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
TermsEnum termsEnum = terms.iterator();
TermsWriter termsWriter = new TermsWriter(fieldInfo);
long sumTotalTermFreq = 0;
long sumDocFreq = 0;
FixedBitSet docsSeen = new FixedBitSet(maxDoc);
while (true) {
BytesRef term = termsEnum.next();
if (term == null) {
break;
}
BlockTermState termState = postingsWriter.writeTerm(term, termsEnum, docsSeen, norms);
if (termState != null) {
termsWriter.finishTerm(term, termState);
sumTotalTermFreq += termState.totalTermFreq;
sumDocFreq += termState.docFreq;
}
}
termsWriter.finish(hasFreq ? sumTotalTermFreq : -1, sumDocFreq, docsSeen.cardinality());
}
}
@Override
public void close() throws IOException {
if (blockOut != null) {
boolean success = false;
try {
final long blockDirStart = blockOut.getFilePointer();
// write field summary
blockOut.writeVInt(fields.size());
for (FieldMetaData field : fields) {
blockOut.writeVInt(field.fieldInfo.number);
blockOut.writeVLong(field.numTerms);
if (field.fieldInfo.getIndexOptions() != IndexOptions.DOCS) {
blockOut.writeVLong(field.sumTotalTermFreq);
}
blockOut.writeVLong(field.sumDocFreq);
blockOut.writeVInt(field.docCount);
blockOut.writeVInt(field.longsSize);
blockOut.writeVLong(field.statsOut.size());
blockOut.writeVLong(field.metaLongsOut.size());
blockOut.writeVLong(field.metaBytesOut.size());
field.skipOut.copyTo(blockOut);
field.statsOut.copyTo(blockOut);
field.metaLongsOut.copyTo(blockOut);
field.metaBytesOut.copyTo(blockOut);
field.dict.save(indexOut);
}
writeTrailer(blockOut, blockDirStart);
CodecUtil.writeFooter(indexOut);
CodecUtil.writeFooter(blockOut);
success = true;
} finally {
if (success) {
IOUtils.close(blockOut, indexOut, postingsWriter);
} else {
IOUtils.closeWhileHandlingException(blockOut, indexOut, postingsWriter);
}
blockOut = null;
}
}
}
private void writeTrailer(IndexOutput out, long dirStart) throws IOException {
out.writeLong(dirStart);
}
private static class FieldMetaData {
public FieldInfo fieldInfo;
public long numTerms;
public long sumTotalTermFreq;
public long sumDocFreq;
public int docCount;
public int longsSize;
public FST<Long> dict;
// TODO: block encode each part
// vint encode next skip point (fully decoded when reading)
public ByteBuffersDataOutput skipOut;
// vint encode df, (ttf-df)
public ByteBuffersDataOutput statsOut;
// vint encode monotonic long[] and length for corresponding byte[]
public ByteBuffersDataOutput metaLongsOut;
// generic byte[]
public ByteBuffersDataOutput metaBytesOut;
}
final class TermsWriter {
private final FSTCompiler<Long> fstCompiler;
private final PositiveIntOutputs outputs;
private final FieldInfo fieldInfo;
private final int longsSize;
private long numTerms;
private final IntsRefBuilder scratchTerm = new IntsRefBuilder();
private final ByteBuffersDataOutput statsOut = new ByteBuffersDataOutput();
private final ByteBuffersDataOutput metaLongsOut = new ByteBuffersDataOutput();
private final ByteBuffersDataOutput metaBytesOut = new ByteBuffersDataOutput();
private final ByteBuffersDataOutput skipOut = new ByteBuffersDataOutput();
private long lastBlockStatsFP;
private long lastBlockMetaLongsFP;
private long lastBlockMetaBytesFP;
private long[] lastBlockLongs;
private long[] lastLongs;
private long lastMetaBytesFP;
TermsWriter(FieldInfo fieldInfo) {
this.numTerms = 0;
this.fieldInfo = fieldInfo;
this.longsSize = postingsWriter.setField(fieldInfo);
this.outputs = PositiveIntOutputs.getSingleton();
this.fstCompiler = new FSTCompiler<>(FST.INPUT_TYPE.BYTE1, outputs);
this.lastBlockStatsFP = 0;
this.lastBlockMetaLongsFP = 0;
this.lastBlockMetaBytesFP = 0;
this.lastBlockLongs = new long[longsSize];
this.lastLongs = new long[longsSize];
this.lastMetaBytesFP = 0;
}
public void finishTerm(BytesRef text, BlockTermState state) throws IOException {
if (numTerms > 0 && numTerms % SKIP_INTERVAL == 0) {
bufferSkip();
}
// write term meta data into fst
final long longs[] = new long[longsSize];
final long delta = state.totalTermFreq - state.docFreq;
if (state.totalTermFreq > 0) {
if (delta == 0) {
statsOut.writeVInt(state.docFreq<<1|1);
} else {
statsOut.writeVInt(state.docFreq<<1);
statsOut.writeVLong(state.totalTermFreq-state.docFreq);
}
} else {
statsOut.writeVInt(state.docFreq);
}
postingsWriter.encodeTerm(longs, metaBytesOut, fieldInfo, state, true);
for (int i = 0; i < longsSize; i++) {
metaLongsOut.writeVLong(longs[i] - lastLongs[i]);
lastLongs[i] = longs[i];
}
metaLongsOut.writeVLong(metaBytesOut.size() - lastMetaBytesFP);
fstCompiler.add(Util.toIntsRef(text, scratchTerm), numTerms);
numTerms++;
lastMetaBytesFP = metaBytesOut.size();
}
public void finish(long sumTotalTermFreq, long sumDocFreq, int docCount) throws IOException {
if (numTerms > 0) {
final FieldMetaData metadata = new FieldMetaData();
metadata.fieldInfo = fieldInfo;
metadata.numTerms = numTerms;
metadata.sumTotalTermFreq = sumTotalTermFreq;
metadata.sumDocFreq = sumDocFreq;
metadata.docCount = docCount;
metadata.longsSize = longsSize;
metadata.skipOut = skipOut;
metadata.statsOut = statsOut;
metadata.metaLongsOut = metaLongsOut;
metadata.metaBytesOut = metaBytesOut;
metadata.dict = fstCompiler.compile();
fields.add(metadata);
}
}
private void bufferSkip() throws IOException {
skipOut.writeVLong(statsOut.size() - lastBlockStatsFP);
skipOut.writeVLong(metaLongsOut.size() - lastBlockMetaLongsFP);
skipOut.writeVLong(metaBytesOut.size() - lastBlockMetaBytesFP);
for (int i = 0; i < longsSize; i++) {
skipOut.writeVLong(lastLongs[i] - lastBlockLongs[i]);
}
lastBlockStatsFP = statsOut.size();
lastBlockMetaLongsFP = metaLongsOut.size();
lastBlockMetaBytesFP = metaBytesOut.size();
System.arraycopy(lastLongs, 0, lastBlockLongs, 0, longsSize);
}
}
}

View File

@ -0,0 +1,78 @@
/*
* 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.codecs.memory;
import java.io.IOException;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsReader;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsWriter;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.IOUtils;
/**
* FST term dict + Lucene50PBF
*/
public final class FSTPostingsFormat extends PostingsFormat {
public FSTPostingsFormat() {
super("FST50");
}
@Override
public String toString() {
return getName();
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase postingsWriter = new Lucene84PostingsWriter(state);
boolean success = false;
try {
FieldsConsumer ret = new FSTTermsWriter(state, postingsWriter);
success = true;
return ret;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(postingsWriter);
}
}
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postingsReader = new Lucene84PostingsReader(state);
boolean success = false;
try {
FieldsProducer ret = new FSTTermsReader(state, postingsReader);
success = true;
return ret;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(postingsReader);
}
}
}
}

View File

@ -0,0 +1,383 @@
/*
* 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.codecs.memory;
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.fst.Outputs;
/**
* An FST {@link Outputs} implementation for
* {@link FSTTermsWriter}.
*
* @lucene.experimental
*/
// NOTE: outputs should be per-field, since
// longsSize is fixed for each field
class FSTTermOutputs extends Outputs<FSTTermOutputs.TermData> {
private final static TermData NO_OUTPUT = new TermData();
//private static boolean TEST = false;
private final boolean hasPos;
private final int longsSize;
/**
* Represents the metadata for one term.
* On an FST, only long[] part is 'shared' and pushed towards root.
* byte[] and term stats will be kept on deeper arcs.
*/
static class TermData implements Accountable {
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(TermData.class);
long[] longs;
byte[] bytes;
int docFreq;
long totalTermFreq;
TermData() {
this.longs = null;
this.bytes = null;
this.docFreq = 0;
this.totalTermFreq = -1;
}
TermData(long[] longs, byte[] bytes, int docFreq, long totalTermFreq) {
this.longs = longs;
this.bytes = bytes;
this.docFreq = docFreq;
this.totalTermFreq = totalTermFreq;
}
@Override
public long ramBytesUsed() {
long ramBytesUsed = BASE_RAM_BYTES_USED;
if (longs != null) {
ramBytesUsed += RamUsageEstimator.sizeOf(longs);
}
if (bytes != null) {
ramBytesUsed += RamUsageEstimator.sizeOf(bytes);
}
return ramBytesUsed;
}
// NOTE: actually, FST nodes are seldom
// identical when outputs on their arcs
// aren't NO_OUTPUTs.
@Override
public int hashCode() {
int hash = 0;
if (longs != null) {
final int end = longs.length;
for (int i = 0; i < end; i++) {
hash -= longs[i];
}
}
if (bytes != null) {
hash = -hash;
final int end = bytes.length;
for (int i = 0; i < end; i++) {
hash += bytes[i];
}
}
hash += docFreq + totalTermFreq;
return hash;
}
@Override
public String toString() {
return "FSTTermOutputs$TermData longs=" + Arrays.toString(longs) + " bytes=" + Arrays.toString(bytes) + " docFreq=" + docFreq + " totalTermFreq=" + totalTermFreq;
}
@Override
public boolean equals(Object other_) {
if (other_ == this) {
return true;
} else if (!(other_ instanceof FSTTermOutputs.TermData)) {
return false;
}
TermData other = (TermData) other_;
return statsEqual(this, other) &&
longsEqual(this, other) &&
bytesEqual(this, other);
}
}
protected FSTTermOutputs(FieldInfo fieldInfo, int longsSize) {
this.hasPos = fieldInfo.getIndexOptions() != IndexOptions.DOCS;
this.longsSize = longsSize;
}
@Override
public long ramBytesUsed(TermData output) {
return output.ramBytesUsed();
}
@Override
//
// The return value will be the smaller one, when these two are
// 'comparable', i.e.
// 1. every value in t1 is not larger than in t2, or
// 2. every value in t1 is not smaller than t2.
//
public TermData common(TermData t1, TermData t2) {
//if (TEST) System.out.print("common("+t1+", "+t2+") = ");
if (t1 == NO_OUTPUT || t2 == NO_OUTPUT) {
//if (TEST) System.out.println("ret:"+NO_OUTPUT);
return NO_OUTPUT;
}
assert t1.longs.length == t2.longs.length;
long[] min = t1.longs, max = t2.longs;
int pos = 0;
TermData ret;
while (pos < longsSize && min[pos] == max[pos]) {
pos++;
}
if (pos < longsSize) { // unequal long[]
if (min[pos] > max[pos]) {
min = t2.longs;
max = t1.longs;
}
// check whether strictly smaller
while (pos < longsSize && min[pos] <= max[pos]) {
pos++;
}
if (pos < longsSize || allZero(min)) { // not comparable or all-zero
ret = NO_OUTPUT;
} else {
ret = new TermData(min, null, 0, -1);
}
} else { // equal long[]
if (statsEqual(t1, t2) && bytesEqual(t1, t2)) {
ret = t1;
} else if (allZero(min)) {
ret = NO_OUTPUT;
} else {
ret = new TermData(min, null, 0, -1);
}
}
//if (TEST) System.out.println("ret:"+ret);
return ret;
}
@Override
public TermData subtract(TermData t1, TermData t2) {
//if (TEST) System.out.print("subtract("+t1+", "+t2+") = ");
if (t2 == NO_OUTPUT) {
//if (TEST) System.out.println("ret:"+t1);
return t1;
}
assert t1.longs.length == t2.longs.length;
int pos = 0;
long diff = 0;
long[] share = new long[longsSize];
while (pos < longsSize) {
share[pos] = t1.longs[pos] - t2.longs[pos];
diff += share[pos];
pos++;
}
TermData ret;
if (diff == 0 && statsEqual(t1, t2) && bytesEqual(t1, t2)) {
ret = NO_OUTPUT;
} else {
ret = new TermData(share, t1.bytes, t1.docFreq, t1.totalTermFreq);
}
//if (TEST) System.out.println("ret:"+ret);
return ret;
}
// TODO: if we refactor a 'addSelf(TermData other)',
// we can gain about 5~7% for fuzzy queries, however this also
// means we are putting too much stress on FST Outputs decoding?
@Override
public TermData add(TermData t1, TermData t2) {
//if (TEST) System.out.print("add("+t1+", "+t2+") = ");
if (t1 == NO_OUTPUT) {
//if (TEST) System.out.println("ret:"+t2);
return t2;
} else if (t2 == NO_OUTPUT) {
//if (TEST) System.out.println("ret:"+t1);
return t1;
}
assert t1.longs.length == t2.longs.length;
int pos = 0;
long[] accum = new long[longsSize];
while (pos < longsSize) {
accum[pos] = t1.longs[pos] + t2.longs[pos];
pos++;
}
TermData ret;
if (t2.bytes != null || t2.docFreq > 0) {
ret = new TermData(accum, t2.bytes, t2.docFreq, t2.totalTermFreq);
} else {
ret = new TermData(accum, t1.bytes, t1.docFreq, t1.totalTermFreq);
}
//if (TEST) System.out.println("ret:"+ret);
return ret;
}
@Override
public void write(TermData data, DataOutput out) throws IOException {
assert hasPos || data.totalTermFreq == -1;
int bit0 = allZero(data.longs) ? 0 : 1;
int bit1 = ((data.bytes == null || data.bytes.length == 0) ? 0 : 1) << 1;
int bit2 = ((data.docFreq == 0) ? 0 : 1) << 2;
int bits = bit0 | bit1 | bit2;
if (bit1 > 0) { // determine extra length
if (data.bytes.length < 32) {
bits |= (data.bytes.length << 3);
out.writeByte((byte)bits);
} else {
out.writeByte((byte)bits);
out.writeVInt(data.bytes.length);
}
} else {
out.writeByte((byte)bits);
}
if (bit0 > 0) { // not all-zero case
for (int pos = 0; pos < longsSize; pos++) {
out.writeVLong(data.longs[pos]);
}
}
if (bit1 > 0) { // bytes exists
out.writeBytes(data.bytes, 0, data.bytes.length);
}
if (bit2 > 0) { // stats exist
if (hasPos) {
if (data.docFreq == data.totalTermFreq) {
out.writeVInt((data.docFreq << 1) | 1);
} else {
out.writeVInt((data.docFreq << 1));
out.writeVLong(data.totalTermFreq - data.docFreq);
}
} else {
out.writeVInt(data.docFreq);
}
}
}
@Override
public TermData read(DataInput in) throws IOException {
long[] longs = new long[longsSize];
byte[] bytes = null;
int docFreq = 0;
long totalTermFreq = -1;
int bits = in.readByte() & 0xff;
int bit0 = bits & 1;
int bit1 = bits & 2;
int bit2 = bits & 4;
int bytesSize = (bits >>> 3);
if (bit1 > 0 && bytesSize == 0) { // determine extra length
bytesSize = in.readVInt();
}
if (bit0 > 0) { // not all-zero case
for (int pos = 0; pos < longsSize; pos++) {
longs[pos] = in.readVLong();
}
}
if (bit1 > 0) { // bytes exists
bytes = new byte[bytesSize];
in.readBytes(bytes, 0, bytesSize);
}
if (bit2 > 0) { // stats exist
int code = in.readVInt();
if (hasPos) {
totalTermFreq = docFreq = code >>> 1;
if ((code & 1) == 0) {
totalTermFreq += in.readVLong();
}
} else {
docFreq = code;
}
}
return new TermData(longs, bytes, docFreq, totalTermFreq);
}
@Override
public void skipOutput(DataInput in) throws IOException {
int bits = in.readByte() & 0xff;
int bit0 = bits & 1;
int bit1 = bits & 2;
int bit2 = bits & 4;
int bytesSize = (bits >>> 3);
if (bit1 > 0 && bytesSize == 0) { // determine extra length
bytesSize = in.readVInt();
}
if (bit0 > 0) { // not all-zero case
for (int pos = 0; pos < longsSize; pos++) {
in.readVLong();
}
}
if (bit1 > 0) { // bytes exists
in.skipBytes(bytesSize);
}
if (bit2 > 0) { // stats exist
int code = in.readVInt();
if (hasPos && (code & 1) == 0) {
in.readVLong();
}
}
}
@Override
public TermData getNoOutput() {
return NO_OUTPUT;
}
@Override
public String outputToString(TermData data) {
return data.toString();
}
static boolean statsEqual(final TermData t1, final TermData t2) {
return t1.docFreq == t2.docFreq && t1.totalTermFreq == t2.totalTermFreq;
}
static boolean bytesEqual(final TermData t1, final TermData t2) {
if (t1.bytes == null && t2.bytes == null) {
return true;
}
return t1.bytes != null && t2.bytes != null && Arrays.equals(t1.bytes, t2.bytes);
}
static boolean longsEqual(final TermData t1, final TermData t2) {
if (t1.longs == null && t2.longs == null) {
return true;
}
return t1.longs != null && t2.longs != null && Arrays.equals(t1.longs, t2.longs);
}
static boolean allZero(final long[] l) {
for (int i = 0; i < l.length; i++) {
if (l[i] != 0) {
return false;
}
}
return true;
}
}

View File

@ -0,0 +1,785 @@
/*
* 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.codecs.memory;
import java.io.IOException;
import java.util.ArrayList;
import java.util.BitSet;
import java.util.Collection;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.TreeMap;
import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.ImpactsEnum;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.TermState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.automaton.ByteRunAutomaton;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.fst.BytesRefFSTEnum;
import org.apache.lucene.util.fst.BytesRefFSTEnum.InputOutput;
import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.Outputs;
import org.apache.lucene.util.fst.Util;
/**
* FST-based terms dictionary reader.
*
* The FST directly maps each term and its metadata,
* it is memory resident.
*
* @lucene.experimental
*/
public class FSTTermsReader extends FieldsProducer {
final TreeMap<String, TermsReader> fields = new TreeMap<>();
final PostingsReaderBase postingsReader;
//static boolean TEST = false;
public FSTTermsReader(SegmentReadState state, PostingsReaderBase postingsReader) throws IOException {
final String termsFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, FSTTermsWriter.TERMS_EXTENSION);
this.postingsReader = postingsReader;
final IndexInput in = state.directory.openInput(termsFileName, state.context);
boolean success = false;
try {
CodecUtil.checkIndexHeader(in, FSTTermsWriter.TERMS_CODEC_NAME,
FSTTermsWriter.TERMS_VERSION_START,
FSTTermsWriter.TERMS_VERSION_CURRENT,
state.segmentInfo.getId(), state.segmentSuffix);
CodecUtil.checksumEntireFile(in);
this.postingsReader.init(in, state);
seekDir(in);
final FieldInfos fieldInfos = state.fieldInfos;
final int numFields = in.readVInt();
for (int i = 0; i < numFields; i++) {
int fieldNumber = in.readVInt();
FieldInfo fieldInfo = fieldInfos.fieldInfo(fieldNumber);
long numTerms = in.readVLong();
long sumTotalTermFreq = in.readVLong();
// if frequencies are omitted, sumTotalTermFreq=sumDocFreq and we only write one value
long sumDocFreq = fieldInfo.getIndexOptions() == IndexOptions.DOCS ? sumTotalTermFreq : in.readVLong();
int docCount = in.readVInt();
int longsSize = in.readVInt();
TermsReader current = new TermsReader(fieldInfo, in, numTerms, sumTotalTermFreq, sumDocFreq, docCount, longsSize);
TermsReader previous = fields.put(fieldInfo.name, current);
checkFieldSummary(state.segmentInfo, in, current, previous);
}
success = true;
} finally {
if (success) {
IOUtils.close(in);
} else {
IOUtils.closeWhileHandlingException(in);
}
}
}
private void seekDir(IndexInput in) throws IOException {
in.seek(in.length() - CodecUtil.footerLength() - 8);
in.seek(in.readLong());
}
private void checkFieldSummary(SegmentInfo info, IndexInput in, TermsReader field, TermsReader previous) throws IOException {
// #docs with field must be <= #docs
if (field.docCount < 0 || field.docCount > info.maxDoc()) {
throw new CorruptIndexException("invalid docCount: " + field.docCount + " maxDoc: " + info.maxDoc(), in);
}
// #postings must be >= #docs with field
if (field.sumDocFreq < field.docCount) {
throw new CorruptIndexException("invalid sumDocFreq: " + field.sumDocFreq + " docCount: " + field.docCount, in);
}
// #positions must be >= #postings
if (field.sumTotalTermFreq < field.sumDocFreq) {
throw new CorruptIndexException("invalid sumTotalTermFreq: " + field.sumTotalTermFreq + " sumDocFreq: " + field.sumDocFreq, in);
}
if (previous != null) {
throw new CorruptIndexException("duplicate fields: " + field.fieldInfo.name, in);
}
}
@Override
public Iterator<String> iterator() {
return Collections.unmodifiableSet(fields.keySet()).iterator();
}
@Override
public Terms terms(String field) throws IOException {
assert field != null;
return fields.get(field);
}
@Override
public int size() {
return fields.size();
}
@Override
public void close() throws IOException {
try {
IOUtils.close(postingsReader);
} finally {
fields.clear();
}
}
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(TermsReader.class);
final class TermsReader extends Terms implements Accountable {
final FieldInfo fieldInfo;
final long numTerms;
final long sumTotalTermFreq;
final long sumDocFreq;
final int docCount;
final int longsSize;
final FST<FSTTermOutputs.TermData> dict;
TermsReader(FieldInfo fieldInfo, IndexInput in, long numTerms, long sumTotalTermFreq, long sumDocFreq, int docCount, int longsSize) throws IOException {
this.fieldInfo = fieldInfo;
this.numTerms = numTerms;
this.sumTotalTermFreq = sumTotalTermFreq;
this.sumDocFreq = sumDocFreq;
this.docCount = docCount;
this.longsSize = longsSize;
this.dict = new FST<>(in, new FSTTermOutputs(fieldInfo, longsSize));
}
@Override
public long ramBytesUsed() {
long bytesUsed = BASE_RAM_BYTES_USED;
if (dict != null) {
bytesUsed += dict.ramBytesUsed();
}
return bytesUsed;
}
@Override
public Collection<Accountable> getChildResources() {
if (dict == null) {
return Collections.emptyList();
} else {
return Collections.singletonList(Accountables.namedAccountable("terms", dict));
}
}
@Override
public String toString() {
return "FSTTerms(terms=" + numTerms + ",postings=" + sumDocFreq + ",positions=" + sumTotalTermFreq + ",docs=" + docCount + ")";
}
@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 fieldInfo.hasPayloads();
}
@Override
public long size() {
return numTerms;
}
@Override
public long getSumTotalTermFreq() {
return sumTotalTermFreq;
}
@Override
public long getSumDocFreq() throws IOException {
return sumDocFreq;
}
@Override
public int getDocCount() throws IOException {
return docCount;
}
@Override
public TermsEnum iterator() throws IOException {
return new SegmentTermsEnum();
}
@Override
public TermsEnum intersect(CompiledAutomaton compiled, BytesRef startTerm) throws IOException {
if (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
}
return new IntersectTermsEnum(compiled, startTerm);
}
// Only wraps common operations for PBF interact
abstract class BaseTermsEnum extends org.apache.lucene.index.BaseTermsEnum {
/* Current term stats + decoded metadata (customized by PBF) */
final BlockTermState state;
/* Current term stats + undecoded metadata (long[] & byte[]) */
FSTTermOutputs.TermData meta;
ByteArrayDataInput bytesReader;
/** Decodes metadata into customized term state */
abstract void decodeMetaData() throws IOException;
BaseTermsEnum() throws IOException {
this.state = postingsReader.newTermState();
this.bytesReader = new ByteArrayDataInput();
// NOTE: metadata will only be initialized in child class
}
@Override
public TermState termState() throws IOException {
decodeMetaData();
return state.clone();
}
@Override
public int docFreq() throws IOException {
return state.docFreq;
}
@Override
public long totalTermFreq() throws IOException {
return state.totalTermFreq == -1 ? state.docFreq : state.totalTermFreq;
}
@Override
public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
decodeMetaData();
return postingsReader.postings(fieldInfo, state, reuse, flags);
}
@Override
public ImpactsEnum impacts(int flags) throws IOException {
decodeMetaData();
return postingsReader.impacts(fieldInfo, state, flags);
}
@Override
public void seekExact(long ord) throws IOException {
throw new UnsupportedOperationException();
}
@Override
public long ord() {
throw new UnsupportedOperationException();
}
}
// Iterates through all terms in this field
private final class SegmentTermsEnum extends BaseTermsEnum {
/* Current term, null when enum ends or unpositioned */
BytesRef term;
final BytesRefFSTEnum<FSTTermOutputs.TermData> fstEnum;
/* True when current term's metadata is decoded */
boolean decoded;
/* True when current enum is 'positioned' by seekExact(TermState) */
boolean seekPending;
SegmentTermsEnum() throws IOException {
super();
this.fstEnum = new BytesRefFSTEnum<>(dict);
this.decoded = false;
this.seekPending = false;
this.meta = null;
}
@Override
public BytesRef term() throws IOException {
return term;
}
// Let PBF decode metadata from long[] and byte[]
@Override
void decodeMetaData() throws IOException {
if (!decoded && !seekPending) {
if (meta.bytes != null) {
bytesReader.reset(meta.bytes, 0, meta.bytes.length);
}
postingsReader.decodeTerm(meta.longs, bytesReader, fieldInfo, state, true);
decoded = true;
}
}
// Update current enum according to FSTEnum
void updateEnum(final InputOutput<FSTTermOutputs.TermData> pair) {
if (pair == null) {
term = null;
} else {
term = pair.input;
meta = pair.output;
state.docFreq = meta.docFreq;
state.totalTermFreq = meta.totalTermFreq;
}
decoded = false;
seekPending = false;
}
@Override
public BytesRef next() throws IOException {
if (seekPending) { // previously positioned, but termOutputs not fetched
seekPending = false;
SeekStatus status = seekCeil(term);
assert status == SeekStatus.FOUND; // must positioned on valid term
}
updateEnum(fstEnum.next());
return term;
}
@Override
public boolean seekExact(BytesRef target) throws IOException {
updateEnum(fstEnum.seekExact(target));
return term != null;
}
@Override
public SeekStatus seekCeil(BytesRef target) throws IOException {
updateEnum(fstEnum.seekCeil(target));
if (term == null) {
return SeekStatus.END;
} else {
return term.equals(target) ? SeekStatus.FOUND : SeekStatus.NOT_FOUND;
}
}
@Override
public void seekExact(BytesRef target, TermState otherState) {
if (!target.equals(term)) {
state.copyFrom(otherState);
term = BytesRef.deepCopyOf(target);
seekPending = true;
}
}
}
// Iterates intersect result with automaton (cannot seek!)
private final class IntersectTermsEnum extends BaseTermsEnum {
/* Current term, null when enum ends or unpositioned */
BytesRefBuilder term;
/* True when current term's metadata is decoded */
boolean decoded;
/* True when there is pending term when calling next() */
boolean pending;
/* stack to record how current term is constructed,
* used to accumulate metadata or rewind term:
* level == term.length + 1,
* == 0 when term is null */
Frame[] stack;
int level;
/* to which level the metadata is accumulated
* so that we can accumulate metadata lazily */
int metaUpto;
/* term dict fst */
final FST<FSTTermOutputs.TermData> fst;
final FST.BytesReader fstReader;
final Outputs<FSTTermOutputs.TermData> fstOutputs;
/* query automaton to intersect with */
final ByteRunAutomaton fsa;
private final class Frame {
/* fst stats */
FST.Arc<FSTTermOutputs.TermData> fstArc;
FSTTermOutputs.TermData output;
/* automaton stats */
int fsaState;
Frame() {
this.fstArc = new FST.Arc<>();
this.fsaState = -1;
}
public String toString() {
return "arc=" + fstArc + " state=" + fsaState;
}
}
IntersectTermsEnum(CompiledAutomaton compiled, BytesRef startTerm) throws IOException {
super();
//if (TEST) System.out.println("Enum init, startTerm=" + startTerm);
this.fst = dict;
this.fstReader = fst.getBytesReader();
this.fstOutputs = dict.outputs;
this.fsa = compiled.runAutomaton;
this.level = -1;
this.stack = new Frame[16];
for (int i = 0 ; i < stack.length; i++) {
this.stack[i] = new Frame();
}
loadVirtualFrame(newFrame());
this.level++;
pushFrame(loadFirstFrame(newFrame()));
this.meta = null;
this.metaUpto = 1;
this.decoded = false;
this.pending = false;
if (startTerm == null) {
pending = isAccept(topFrame());
} else {
doSeekCeil(startTerm);
pending = (term == null || !startTerm.equals(term.get())) && isValid(topFrame()) && isAccept(topFrame());
}
}
@Override
public BytesRef term() throws IOException {
return term == null ? null : term.get();
}
@Override
void decodeMetaData() throws IOException {
assert term != null;
if (!decoded) {
if (meta.bytes != null) {
bytesReader.reset(meta.bytes, 0, meta.bytes.length);
}
postingsReader.decodeTerm(meta.longs, bytesReader, fieldInfo, state, true);
decoded = true;
}
}
/** Lazily accumulate meta data, when we got a accepted term */
void loadMetaData() {
Frame last, next;
last = stack[metaUpto];
while (metaUpto != level) {
metaUpto++;
next = stack[metaUpto];
next.output = fstOutputs.add(next.output, last.output);
last = next;
}
if (last.fstArc.isFinal()) {
meta = fstOutputs.add(last.output, last.fstArc.nextFinalOutput());
} else {
meta = last.output;
}
state.docFreq = meta.docFreq;
state.totalTermFreq = meta.totalTermFreq;
}
@Override
public SeekStatus seekCeil(BytesRef target) throws IOException {
decoded = false;
doSeekCeil(target);
loadMetaData();
if (term == null) {
return SeekStatus.END;
} else {
return term.equals(target) ? SeekStatus.FOUND : SeekStatus.NOT_FOUND;
}
}
@Override
public BytesRef next() throws IOException {
//if (TEST) System.out.println("Enum next()");
if (pending) {
pending = false;
loadMetaData();
return term();
}
decoded = false;
DFS:
while (level > 0) {
Frame frame = newFrame();
if (loadExpandFrame(topFrame(), frame) != null) { // has valid target
pushFrame(frame);
if (isAccept(frame)) { // gotcha
break;
}
continue; // check next target
}
frame = popFrame();
while(level > 0) {
if (loadNextFrame(topFrame(), frame) != null) { // has valid sibling
pushFrame(frame);
if (isAccept(frame)) { // gotcha
break DFS;
}
continue DFS; // check next target
}
frame = popFrame();
}
return null;
}
loadMetaData();
return term();
}
private BytesRef doSeekCeil(BytesRef target) throws IOException {
//if (TEST) System.out.println("Enum doSeekCeil()");
Frame frame= null;
int label, upto = 0, limit = target.length;
while (upto < limit) { // to target prefix, or ceil label (rewind prefix)
frame = newFrame();
label = target.bytes[upto] & 0xff;
frame = loadCeilFrame(label, topFrame(), frame);
if (frame == null || frame.fstArc.label() != label) {
break;
}
assert isValid(frame); // target must be fetched from automaton
pushFrame(frame);
upto++;
}
if (upto == limit) { // got target
return term();
}
if (frame != null) { // got larger term('s prefix)
pushFrame(frame);
return isAccept(frame) ? term() : next();
}
while (level > 0) { // got target's prefix, advance to larger term
frame = popFrame();
while (level > 0 && !canRewind(frame)) {
frame = popFrame();
}
if (loadNextFrame(topFrame(), frame) != null) {
pushFrame(frame);
return isAccept(frame) ? term() : next();
}
}
return null;
}
/** Virtual frame, never pop */
Frame loadVirtualFrame(Frame frame) {
frame.output = fstOutputs.getNoOutput();
frame.fsaState = -1;
return frame;
}
/** Load frame for start arc(node) on fst */
Frame loadFirstFrame(Frame frame) throws IOException {
frame.fstArc = fst.getFirstArc(frame.fstArc);
frame.output = frame.fstArc.output();
frame.fsaState = 0;
return frame;
}
/** Load frame for target arc(node) on fst */
Frame loadExpandFrame(Frame top, Frame frame) throws IOException {
if (!canGrow(top)) {
return null;
}
frame.fstArc = fst.readFirstRealTargetArc(top.fstArc.target(), frame.fstArc, fstReader);
frame.fsaState = fsa.step(top.fsaState, frame.fstArc.label());
//if (TEST) System.out.println(" loadExpand frame="+frame);
if (frame.fsaState == -1) {
return loadNextFrame(top, frame);
}
frame.output = frame.fstArc.output();
return frame;
}
/** Load frame for sibling arc(node) on fst */
Frame loadNextFrame(Frame top, Frame frame) throws IOException {
if (!canRewind(frame)) {
return null;
}
while (!frame.fstArc.isLast()) {
frame.fstArc = fst.readNextRealArc(frame.fstArc, fstReader);
frame.fsaState = fsa.step(top.fsaState, frame.fstArc.label());
if (frame.fsaState != -1) {
break;
}
}
//if (TEST) System.out.println(" loadNext frame="+frame);
if (frame.fsaState == -1) {
return null;
}
frame.output = frame.fstArc.output();
return frame;
}
/** Load frame for target arc(node) on fst, so that
* arc.label &gt;= label and !fsa.reject(arc.label) */
Frame loadCeilFrame(int label, Frame top, Frame frame) throws IOException {
FST.Arc<FSTTermOutputs.TermData> arc = frame.fstArc;
arc = Util.readCeilArc(label, fst, top.fstArc, arc, fstReader);
if (arc == null) {
return null;
}
frame.fsaState = fsa.step(top.fsaState, arc.label());
//if (TEST) System.out.println(" loadCeil frame="+frame);
if (frame.fsaState == -1) {
return loadNextFrame(top, frame);
}
frame.output = frame.fstArc.output();
return frame;
}
boolean isAccept(Frame frame) { // reach a term both fst&fsa accepts
return fsa.isAccept(frame.fsaState) && frame.fstArc.isFinal();
}
boolean isValid(Frame frame) { // reach a prefix both fst&fsa won't reject
return /*frame != null &&*/ frame.fsaState != -1;
}
boolean canGrow(Frame frame) { // can walk forward on both fst&fsa
return frame.fsaState != -1 && FST.targetHasArcs(frame.fstArc);
}
boolean canRewind(Frame frame) { // can jump to sibling
return !frame.fstArc.isLast();
}
void pushFrame(Frame frame) {
term = grow(frame.fstArc.label());
level++;
//if (TEST) System.out.println(" term=" + term + " level=" + level);
}
Frame popFrame() {
term = shrink();
level--;
metaUpto = metaUpto > level ? level : metaUpto;
//if (TEST) System.out.println(" term=" + term + " level=" + level);
return stack[level+1];
}
Frame newFrame() {
if (level+1 == stack.length) {
final Frame[] temp = new Frame[ArrayUtil.oversize(level+2, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
System.arraycopy(stack, 0, temp, 0, stack.length);
for (int i = stack.length; i < temp.length; i++) {
temp[i] = new Frame();
}
stack = temp;
}
return stack[level+1];
}
Frame topFrame() {
return stack[level];
}
BytesRefBuilder grow(int label) {
if (term == null) {
term = new BytesRefBuilder();
} else {
term.append((byte)label);
}
return term;
}
BytesRefBuilder shrink() {
if (term.length() == 0) {
term = null;
} else {
term.setLength(term.length() - 1);
}
return term;
}
}
}
static<T> void walk(FST<T> fst) throws IOException {
final ArrayList<FST.Arc<T>> queue = new ArrayList<>();
final BitSet seen = new BitSet();
final FST.BytesReader reader = fst.getBytesReader();
final FST.Arc<T> startArc = fst.getFirstArc(new FST.Arc<T>());
queue.add(startArc);
while (!queue.isEmpty()) {
final FST.Arc<T> arc = queue.remove(0);
final long node = arc.target();
//System.out.println(arc);
if (FST.targetHasArcs(arc) && !seen.get((int) node)) {
seen.set((int) node);
fst.readFirstRealTargetArc(node, arc, reader);
while (true) {
queue.add(new FST.Arc<T>().copyFrom(arc));
if (arc.isLast()) {
break;
} else {
fst.readNextRealArc(arc, reader);
}
}
}
}
}
@Override
public long ramBytesUsed() {
long ramBytesUsed = postingsReader.ramBytesUsed();
for (TermsReader r : fields.values()) {
ramBytesUsed += r.ramBytesUsed();
}
return ramBytesUsed;
}
@Override
public Collection<Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>(Accountables.namedAccountables("field", fields));
resources.add(Accountables.namedAccountable("delegate", postingsReader));
return Collections.unmodifiableCollection(resources);
}
@Override
public String toString() {
return getClass().getSimpleName() + "(fields=" + fields.size() + ",delegate=" + postingsReader + ")";
}
@Override
public void checkIntegrity() throws IOException {
postingsReader.checkIntegrity();
}
}

View File

@ -0,0 +1,291 @@
/*
* 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.codecs.memory;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.IntsRefBuilder;
import org.apache.lucene.util.fst.FSTCompiler;
import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.Util;
/**
* FST-based term dict, using metadata as FST output.
*
* The FST directly holds the mapping between &lt;term, metadata&gt;.
*
* Term metadata consists of three parts:
* 1. term statistics: docFreq, totalTermFreq;
* 2. monotonic long[], e.g. the pointer to the postings list for that term;
* 3. generic byte[], e.g. other information need by postings reader.
*
* <p>
* File:
* <ul>
* <li><tt>.tst</tt>: <a href="#Termdictionary">Term Dictionary</a></li>
* </ul>
* <p>
*
* <a name="Termdictionary"></a>
* <h3>Term Dictionary</h3>
* <p>
* The .tst contains a list of FSTs, one for each field.
* The FST maps a term to its corresponding statistics (e.g. docfreq)
* and metadata (e.g. information for postings list reader like file pointer
* to postings list).
* </p>
* <p>
* Typically the metadata is separated into two parts:
* <ul>
* <li>
* Monotonical long array: Some metadata will always be ascending in order
* with the corresponding term. This part is used by FST to share outputs between arcs.
* </li>
* <li>
* Generic byte array: Used to store non-monotonic metadata.
* </li>
* </ul>
*
* File format:
* <ul>
* <li>TermsDict(.tst) --&gt; Header, <i>PostingsHeader</i>, FieldSummary, DirOffset</li>
* <li>FieldSummary --&gt; NumFields, &lt;FieldNumber, NumTerms, SumTotalTermFreq?,
* SumDocFreq, DocCount, LongsSize, TermFST &gt;<sup>NumFields</sup></li>
* <li>TermFST --&gt; {@link FST FST&lt;TermData&gt;}</li>
* <li>TermData --&gt; Flag, BytesSize?, LongDelta<sup>LongsSize</sup>?, Byte<sup>BytesSize</sup>?,
* &lt; DocFreq[Same?], (TotalTermFreq-DocFreq) &gt; ? </li>
* <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
* <li>DirOffset --&gt; {@link DataOutput#writeLong Uint64}</li>
* <li>DocFreq, LongsSize, BytesSize, NumFields,
* FieldNumber, DocCount --&gt; {@link DataOutput#writeVInt VInt}</li>
* <li>TotalTermFreq, NumTerms, SumTotalTermFreq, SumDocFreq, LongDelta --&gt;
* {@link DataOutput#writeVLong VLong}</li>
* </ul>
* <p>Notes:</p>
* <ul>
* <li>
* The format of PostingsHeader and generic meta bytes are customized by the specific postings implementation:
* they contain arbitrary per-file data (such as parameters or versioning information), and per-term data
* (non-monotonic ones like pulsed postings data).
* </li>
* <li>
* The format of TermData is determined by FST, typically monotonic metadata will be dense around shallow arcs,
* while in deeper arcs only generic bytes and term statistics exist.
* </li>
* <li>
* The byte Flag is used to indicate which part of metadata exists on current arc. Specially the monotonic part
* is omitted when it is an array of 0s.
* </li>
* <li>
* Since LongsSize is per-field fixed, it is only written once in field summary.
* </li>
* </ul>
*
* @lucene.experimental
*/
public class FSTTermsWriter extends FieldsConsumer {
static final String TERMS_EXTENSION = "tfp";
static final String TERMS_CODEC_NAME = "FSTTerms";
public static final int TERMS_VERSION_START = 2;
public static final int TERMS_VERSION_CURRENT = TERMS_VERSION_START;
final PostingsWriterBase postingsWriter;
final FieldInfos fieldInfos;
IndexOutput out;
final int maxDoc;
final List<FieldMetaData> fields = new ArrayList<>();
public FSTTermsWriter(SegmentWriteState state, PostingsWriterBase postingsWriter) throws IOException {
final String termsFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_EXTENSION);
this.postingsWriter = postingsWriter;
this.fieldInfos = state.fieldInfos;
this.out = state.directory.createOutput(termsFileName, state.context);
this.maxDoc = state.segmentInfo.maxDoc();
boolean success = false;
try {
CodecUtil.writeIndexHeader(out, TERMS_CODEC_NAME, TERMS_VERSION_CURRENT,
state.segmentInfo.getId(), state.segmentSuffix);
this.postingsWriter.init(out, state);
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(out);
}
}
}
private void writeTrailer(IndexOutput out, long dirStart) throws IOException {
out.writeLong(dirStart);
}
@Override
public void write(Fields fields, NormsProducer norms) throws IOException {
for(String field : fields) {
Terms terms = fields.terms(field);
if (terms == null) {
continue;
}
FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
boolean hasFreq = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
TermsEnum termsEnum = terms.iterator();
TermsWriter termsWriter = new TermsWriter(fieldInfo);
long sumTotalTermFreq = 0;
long sumDocFreq = 0;
FixedBitSet docsSeen = new FixedBitSet(maxDoc);
while (true) {
BytesRef term = termsEnum.next();
if (term == null) {
break;
}
BlockTermState termState = postingsWriter.writeTerm(term, termsEnum, docsSeen, norms);
if (termState != null) {
termsWriter.finishTerm(term, termState);
sumTotalTermFreq += termState.totalTermFreq;
sumDocFreq += termState.docFreq;
}
}
termsWriter.finish(hasFreq ? sumTotalTermFreq : -1, sumDocFreq, docsSeen.cardinality());
}
}
@Override
public void close() throws IOException {
if (out != null) {
boolean success = false;
try {
// write field summary
final long dirStart = out.getFilePointer();
out.writeVInt(fields.size());
for (FieldMetaData field : fields) {
out.writeVInt(field.fieldInfo.number);
out.writeVLong(field.numTerms);
if (field.fieldInfo.getIndexOptions() != IndexOptions.DOCS) {
out.writeVLong(field.sumTotalTermFreq);
}
out.writeVLong(field.sumDocFreq);
out.writeVInt(field.docCount);
out.writeVInt(field.longsSize);
field.dict.save(out);
}
writeTrailer(out, dirStart);
CodecUtil.writeFooter(out);
success = true;
} finally {
if (success) {
IOUtils.close(out, postingsWriter);
} else {
IOUtils.closeWhileHandlingException(out, postingsWriter);
}
out = null;
}
}
}
private static class FieldMetaData {
public final FieldInfo fieldInfo;
public final long numTerms;
public final long sumTotalTermFreq;
public final long sumDocFreq;
public final int docCount;
public final int longsSize;
public final FST<FSTTermOutputs.TermData> dict;
public FieldMetaData(FieldInfo fieldInfo, long numTerms, long sumTotalTermFreq, long sumDocFreq, int docCount, int longsSize, FST<FSTTermOutputs.TermData> fst) {
this.fieldInfo = fieldInfo;
this.numTerms = numTerms;
this.sumTotalTermFreq = sumTotalTermFreq;
this.sumDocFreq = sumDocFreq;
this.docCount = docCount;
this.longsSize = longsSize;
this.dict = fst;
}
}
final class TermsWriter {
private final FSTCompiler<FSTTermOutputs.TermData> fstCompiler;
private final FSTTermOutputs outputs;
private final FieldInfo fieldInfo;
private final int longsSize;
private long numTerms;
private final IntsRefBuilder scratchTerm = new IntsRefBuilder();
private final ByteBuffersDataOutput metaWriter = ByteBuffersDataOutput.newResettableInstance();
TermsWriter(FieldInfo fieldInfo) {
this.numTerms = 0;
this.fieldInfo = fieldInfo;
this.longsSize = postingsWriter.setField(fieldInfo);
this.outputs = new FSTTermOutputs(fieldInfo, longsSize);
this.fstCompiler = new FSTCompiler<>(FST.INPUT_TYPE.BYTE1, outputs);
}
public void finishTerm(BytesRef text, BlockTermState state) throws IOException {
// write term meta data into fst
final FSTTermOutputs.TermData meta = new FSTTermOutputs.TermData();
meta.longs = new long[longsSize];
meta.bytes = null;
meta.docFreq = state.docFreq;
meta.totalTermFreq = state.totalTermFreq;
postingsWriter.encodeTerm(meta.longs, metaWriter, fieldInfo, state, true);
if (metaWriter.size() > 0) {
meta.bytes = metaWriter.toArrayCopy();
metaWriter.reset();
}
fstCompiler.add(Util.toIntsRef(text, scratchTerm), meta);
numTerms++;
}
public void finish(long sumTotalTermFreq, long sumDocFreq, int docCount) throws IOException {
// save FST dict
if (numTerms > 0) {
final FST<FSTTermOutputs.TermData> fst = fstCompiler.compile();
fields.add(new FieldMetaData(fieldInfo, numTerms, sumTotalTermFreq, sumDocFreq, docCount, longsSize, fst));
}
}
}
}

View File

@ -94,7 +94,7 @@ public class DeltaBaseTermStateSerializer implements Accountable {
/**
* Writes a {@link BlockTermState} to the provided {@link DataOutput}.
* <p>
* Simpler variant of {@link Lucene84PostingsWriter#encodeTerm(DataOutput, FieldInfo, BlockTermState, boolean)}.
* Simpler variant of {@link Lucene84PostingsWriter#encodeTerm(long[], DataOutput, FieldInfo, BlockTermState, boolean)}.
*/
public void writeTermState(DataOutput termStatesOutput, FieldInfo fieldInfo, BlockTermState termState) throws IOException {
IndexOptions indexOptions = fieldInfo.getIndexOptions();
@ -143,7 +143,7 @@ public class DeltaBaseTermStateSerializer implements Accountable {
/**
* Reads a {@link BlockTermState} from the provided {@link DataInput}.
* <p>
* Simpler variant of {@link Lucene84PostingsReader#decodeTerm(DataInput, FieldInfo, BlockTermState, boolean)}.
* Simpler variant of {@link Lucene84PostingsReader#decodeTerm(long[], DataInput, FieldInfo, BlockTermState, boolean)}.
*
* @param reuse {@link BlockTermState} to reuse; or null to create a new one.
*/

View File

@ -16,5 +16,7 @@
org.apache.lucene.codecs.blocktreeords.BlockTreeOrdsPostingsFormat
org.apache.lucene.codecs.bloom.BloomFilteringPostingsFormat
org.apache.lucene.codecs.memory.DirectPostingsFormat
org.apache.lucene.codecs.memory.FSTOrdPostingsFormat
org.apache.lucene.codecs.memory.FSTPostingsFormat
org.apache.lucene.codecs.uniformsplit.UniformSplitPostingsFormat
org.apache.lucene.codecs.uniformsplit.sharedterms.STUniformSplitPostingsFormat

View File

@ -0,0 +1,34 @@
/*
* 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.codecs.memory;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.BasePostingsFormatTestCase;
import org.apache.lucene.util.TestUtil;
/**
* Tests FSTOrdPostingsFormat
*/
public class TestFSTOrdPostingsFormat extends BasePostingsFormatTestCase {
private final Codec codec = TestUtil.alwaysPostingsFormat(new FSTOrdPostingsFormat());
@Override
protected Codec getCodec() {
return codec;
}
}

View File

@ -0,0 +1,34 @@
/*
* 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.codecs.memory;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.BasePostingsFormatTestCase;
import org.apache.lucene.util.TestUtil;
/**
* Tests FSTPostingsFormat
*/
public class TestFSTPostingsFormat extends BasePostingsFormatTestCase {
private final Codec codec = TestUtil.alwaysPostingsFormat(new FSTPostingsFormat());
@Override
protected Codec getCodec() {
return codec;
}
}

View File

@ -159,7 +159,7 @@ public class TestTermBytesComparator extends LuceneTestCase {
}
@Override
public void decodeTerm(DataInput in, FieldInfo fieldInfo, BlockTermState state, boolean absolute) {
public void decodeTerm(long[] longs, DataInput in, FieldInfo fieldInfo, BlockTermState state, boolean absolute) {
}
@Override

View File

@ -268,7 +268,7 @@ public class STBlockReaderTest extends LuceneTestCase {
}
@Override
public void decodeTerm(DataInput in, FieldInfo fieldInfo, BlockTermState state, boolean absolute) {
public void decodeTerm(long[] longs, DataInput in, FieldInfo fieldInfo, BlockTermState state, boolean absolute) {
}
@Override

View File

@ -61,7 +61,7 @@ public abstract class PostingsReaderBase implements Closeable, Accountable {
/** Actually decode metadata for next term
* @see PostingsWriterBase#encodeTerm
*/
public abstract void decodeTerm(DataInput in, FieldInfo fieldInfo, BlockTermState state, boolean absolute) throws IOException;
public abstract void decodeTerm(long[] longs, DataInput in, FieldInfo fieldInfo, BlockTermState state, boolean absolute) throws IOException;
/** Must fully consume state, since after this call that
* TermState may be reused. */

View File

@ -68,12 +68,21 @@ public abstract class PostingsWriterBase implements Closeable {
* Usually elements in {@code longs} are file pointers, so each one always
* increases when a new term is consumed. {@code out} is used to write generic
* bytes, which are not monotonic.
*
* NOTE: sometimes long[] might contain "don't care" values that are unused, e.g.
* the pointer to postings list may not be defined for some terms but is defined
* for others, if it is designed to inline some postings data in term dictionary.
* In this case, the postings writer should always use the last value, so that each
* element in metadata long[] remains monotonic.
*/
public abstract void encodeTerm(DataOutput out, FieldInfo fieldInfo, BlockTermState state, boolean absolute) throws IOException;
public abstract void encodeTerm(long[] longs, DataOutput out, FieldInfo fieldInfo, BlockTermState state, boolean absolute) throws IOException;
/**
* Sets the current field for writing. */
public abstract void setField(FieldInfo fieldInfo);
* Sets the current field for writing, and returns the
* fixed length of long[] metadata (which is fixed per
* field), called when the writing switches to another field. */
// TODO: better name?
public abstract int setField(FieldInfo fieldInfo);
@Override
public abstract void close() throws IOException;

View File

@ -87,7 +87,7 @@ public abstract class PushPostingsWriterBase extends PostingsWriterBase {
* fixed length of long[] metadata (which is fixed per
* field), called when the writing switches to another field. */
@Override
public void setField(FieldInfo fieldInfo) {
public int setField(FieldInfo fieldInfo) {
this.fieldInfo = fieldInfo;
indexOptions = fieldInfo.getIndexOptions();
@ -113,6 +113,8 @@ public abstract class PushPostingsWriterBase extends PostingsWriterBase {
enumFlags = PostingsEnum.OFFSETS;
}
}
return 0;
}
@Override

View File

@ -128,11 +128,8 @@ public final class BlockTreeTermsReader extends FieldsProducer {
/** Auto-prefix terms have been superseded by points. */
public static final int VERSION_AUTO_PREFIX_TERMS_REMOVED = 3;
/** The long[] + byte[] metadata has been replaced with a single byte[]. */
public static final int VERSION_META_LONGS_REMOVED = 4;
/** Current terms format. */
public static final int VERSION_CURRENT = VERSION_META_LONGS_REMOVED;
public static final int VERSION_CURRENT = VERSION_AUTO_PREFIX_TERMS_REMOVED;
/** Extension of terms index file */
static final String TERMS_INDEX_EXTENSION = "tip";
@ -215,11 +212,9 @@ public final class BlockTreeTermsReader extends FieldsProducer {
// when frequencies are omitted, sumDocFreq=sumTotalTermFreq and only one value is written.
final long sumDocFreq = fieldInfo.getIndexOptions() == IndexOptions.DOCS ? sumTotalTermFreq : termsIn.readVLong();
final int docCount = termsIn.readVInt();
if (version < VERSION_META_LONGS_REMOVED) {
final int longsSize = termsIn.readVInt();
if (longsSize < 0) {
throw new CorruptIndexException("invalid longsSize for field: " + fieldInfo.name + ", longsSize=" + longsSize, termsIn);
}
final int longsSize = termsIn.readVInt();
if (longsSize < 0) {
throw new CorruptIndexException("invalid longsSize for field: " + fieldInfo.name + ", longsSize=" + longsSize, termsIn);
}
BytesRef minTerm = readBytesRef(termsIn);
BytesRef maxTerm = readBytesRef(termsIn);
@ -236,7 +231,7 @@ public final class BlockTreeTermsReader extends FieldsProducer {
final long indexStartFP = indexIn.readVLong();
FieldReader previous = fieldMap.put(fieldInfo.name,
new FieldReader(this, fieldInfo, numTerms, rootCode, sumTotalTermFreq, sumDocFreq, docCount,
indexStartFP, indexIn, minTerm, maxTerm, state.openedFromWriter, perFieldLoadMode));
indexStartFP, longsSize, indexIn, minTerm, maxTerm, state.openedFromWriter, perFieldLoadMode));
if (previous != null) {
throw new CorruptIndexException("duplicate field: " + fieldInfo.name, termsIn);
}

View File

@ -224,10 +224,11 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
public final long sumTotalTermFreq;
public final long sumDocFreq;
public final int docCount;
private final int longsSize;
public final BytesRef minTerm;
public final BytesRef maxTerm;
public FieldMetaData(FieldInfo fieldInfo, BytesRef rootCode, long numTerms, long indexStartFP, long sumTotalTermFreq, long sumDocFreq, int docCount,
public FieldMetaData(FieldInfo fieldInfo, BytesRef rootCode, long numTerms, long indexStartFP, long sumTotalTermFreq, long sumDocFreq, int docCount, int longsSize,
BytesRef minTerm, BytesRef maxTerm) {
assert numTerms > 0;
this.fieldInfo = fieldInfo;
@ -238,6 +239,7 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
this.sumTotalTermFreq = sumTotalTermFreq;
this.sumDocFreq = sumDocFreq;
this.docCount = docCount;
this.longsSize = longsSize;
this.minTerm = minTerm;
this.maxTerm = maxTerm;
}
@ -507,6 +509,7 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
class TermsWriter {
private final FieldInfo fieldInfo;
private final int longsSize;
private long numTerms;
final FixedBitSet docsSeen;
long sumTotalTermFreq;
@ -521,6 +524,8 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
private final BytesRefBuilder lastTerm = new BytesRefBuilder();
private int[] prefixStarts = new int[8];
private final long[] longs;
// Pending stack of terms and blocks. As terms arrive (in sorted order)
// we append to this stack, and once the top of the stack has enough
// terms starting with a common prefix, we write a new block with
@ -715,7 +720,13 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
}
// Write term meta data
postingsWriter.encodeTerm(metaWriter, fieldInfo, state, absolute);
postingsWriter.encodeTerm(longs, bytesWriter, fieldInfo, state, absolute);
for (int pos = 0; pos < longsSize; pos++) {
assert longs[pos] >= 0;
metaWriter.writeVLong(longs[pos]);
}
bytesWriter.copyTo(metaWriter);
bytesWriter.reset();
absolute = false;
}
} else {
@ -760,7 +771,13 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
// separate anymore:
// Write term meta data
postingsWriter.encodeTerm(metaWriter, fieldInfo, state, absolute);
postingsWriter.encodeTerm(longs, bytesWriter, fieldInfo, state, absolute);
for (int pos = 0; pos < longsSize; pos++) {
assert longs[pos] >= 0;
metaWriter.writeVLong(longs[pos]);
}
bytesWriter.copyTo(metaWriter);
bytesWriter.reset();
absolute = false;
} else {
PendingBlock block = (PendingBlock) ent;
@ -828,7 +845,9 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
this.fieldInfo = fieldInfo;
assert fieldInfo.getIndexOptions() != IndexOptions.NONE;
docsSeen = new FixedBitSet(maxDoc);
postingsWriter.setField(fieldInfo);
this.longsSize = postingsWriter.setField(fieldInfo);
this.longs = new long[longsSize];
}
/** Writes one term's worth of postings. */
@ -945,6 +964,7 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
sumTotalTermFreq,
sumDocFreq,
docsSeen.cardinality(),
longsSize,
minTerm, maxTerm));
} else {
assert sumTotalTermFreq == 0 || fieldInfo.getIndexOptions() == IndexOptions.DOCS && sumTotalTermFreq == -1;
@ -956,6 +976,7 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
private final ByteBuffersDataOutput suffixWriter = ByteBuffersDataOutput.newResettableInstance();
private final ByteBuffersDataOutput statsWriter = ByteBuffersDataOutput.newResettableInstance();
private final ByteBuffersDataOutput metaWriter = ByteBuffersDataOutput.newResettableInstance();
private final ByteBuffersDataOutput bytesWriter = ByteBuffersDataOutput.newResettableInstance();
}
private boolean closed;
@ -988,6 +1009,7 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
}
termsOut.writeVLong(field.sumDocFreq);
termsOut.writeVInt(field.docCount);
termsOut.writeVInt(field.longsSize);
indexOut.writeVLong(field.indexStartFP);
writeBytesRef(termsOut, field.minTerm);
writeBytesRef(termsOut, field.maxTerm);

View File

@ -58,6 +58,7 @@ public final class FieldReader extends Terms implements Accountable {
final BytesRef rootCode;
final BytesRef minTerm;
final BytesRef maxTerm;
final int longsSize;
final BlockTreeTermsReader parent;
final FST<BytesRef> index;
@ -65,7 +66,7 @@ public final class FieldReader extends Terms implements Accountable {
//private boolean DEBUG;
FieldReader(BlockTreeTermsReader parent, FieldInfo fieldInfo, long numTerms, BytesRef rootCode, long sumTotalTermFreq, long sumDocFreq, int docCount,
long indexStartFP, IndexInput indexIn, BytesRef minTerm, BytesRef maxTerm, boolean openedFromWriter, BlockTreeTermsReader.FSTLoadMode fstLoadMode) throws IOException {
long indexStartFP, int longsSize, IndexInput indexIn, BytesRef minTerm, BytesRef maxTerm, boolean openedFromWriter, BlockTreeTermsReader.FSTLoadMode fstLoadMode) throws IOException {
assert numTerms > 0;
this.fieldInfo = fieldInfo;
//DEBUG = BlockTreeTermsReader.DEBUG && fieldInfo.name.equals("id");
@ -76,6 +77,7 @@ public final class FieldReader extends Terms implements Accountable {
this.docCount = docCount;
this.indexStartFP = indexStartFP;
this.rootCode = rootCode;
this.longsSize = longsSize;
this.minTerm = minTerm;
this.maxTerm = maxTerm;
// if (DEBUG) {

View File

@ -80,8 +80,11 @@ final class IntersectTermsEnumFrame {
FST.Arc<BytesRef> arc;
final BlockTermState termState;
// metadata buffer, holding monotonic values
final long[] longs;
// metadata buffer
// metadata buffer, holding general values
byte[] bytes = new byte[32];
final ByteArrayDataInput bytesReader = new ByteArrayDataInput();
@ -99,6 +102,7 @@ final class IntersectTermsEnumFrame {
this.ord = ord;
this.termState = ite.fr.parent.postingsReader.newTermState();
this.termState.totalTermFreq = -1;
this.longs = new long[ite.fr.longsSize];
}
void loadNextFloorBlock() throws IOException {
@ -274,8 +278,11 @@ final class IntersectTermsEnumFrame {
} else {
termState.totalTermFreq = termState.docFreq + statsReader.readVLong();
}
// metadata
ite.fr.parent.postingsReader.decodeTerm(bytesReader, ite.fr.fieldInfo, termState, absolute);
// metadata
for (int i = 0; i < ite.fr.longsSize; i++) {
longs[i] = bytesReader.readVLong();
}
ite.fr.parent.postingsReader.decodeTerm(longs, bytesReader, ite.fr.fieldInfo, termState, absolute);
metaDataUpto++;
absolute = false;

View File

@ -85,7 +85,9 @@ final class SegmentTermsEnumFrame {
final BlockTermState state;
// metadata buffer
// metadata buffer, holding monotonic values
final long[] longs;
// metadata buffer, holding general values
byte[] bytes = new byte[32];
final ByteArrayDataInput bytesReader = new ByteArrayDataInput();
@ -96,6 +98,7 @@ final class SegmentTermsEnumFrame {
this.ord = ord;
this.state = ste.fr.parent.postingsReader.newTermState();
this.state.totalTermFreq = -1;
this.longs = new long[ste.fr.longsSize];
}
public void setFloorData(ByteArrayDataInput in, BytesRef source) {
@ -421,8 +424,11 @@ final class SegmentTermsEnumFrame {
state.totalTermFreq = state.docFreq + statsReader.readVLong();
//if (DEBUG) System.out.println(" totTF=" + state.totalTermFreq);
}
// metadata
ste.fr.parent.postingsReader.decodeTerm(bytesReader, ste.fr.fieldInfo, state, absolute);
// metadata
for (int i = 0; i < ste.fr.longsSize; i++) {
longs[i] = bytesReader.readVLong();
}
ste.fr.parent.postingsReader.decodeTerm(longs, bytesReader, ste.fr.fieldInfo, state, absolute);
metaDataUpto++;
absolute = false;

View File

@ -166,7 +166,7 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
}
@Override
public void decodeTerm(DataInput in, FieldInfo fieldInfo, BlockTermState _termState, boolean absolute)
public void decodeTerm(long[] longs, DataInput in, FieldInfo fieldInfo, BlockTermState _termState, boolean absolute)
throws IOException {
final IntBlockTermState termState = (IntBlockTermState) _termState;
final boolean fieldHasPositions = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
@ -179,11 +179,11 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
termState.payStartFP = 0;
}
termState.docStartFP += in.readVLong();
termState.docStartFP += longs[0];
if (fieldHasPositions) {
termState.posStartFP += in.readVLong();
termState.posStartFP += longs[1];
if (fieldHasOffsets || fieldHasPayloads) {
termState.payStartFP += in.readVLong();
termState.payStartFP += longs[2];
}
}
if (termState.docFreq == 1) {

View File

@ -190,11 +190,20 @@ public final class Lucene84PostingsWriter extends PushPostingsWriterBase {
}
@Override
public void setField(FieldInfo fieldInfo) {
public int setField(FieldInfo fieldInfo) {
super.setField(fieldInfo);
skipWriter.setField(writePositions, writeOffsets, writePayloads);
lastState = emptyState;
fieldHasNorms = fieldInfo.hasNorms();
if (writePositions) {
if (writePayloads || writeOffsets) {
return 3; // doc + pos + pay FP
} else {
return 2; // doc + pos FP
}
} else {
return 1; // doc FP
}
}
@Override
@ -457,16 +466,16 @@ public final class Lucene84PostingsWriter extends PushPostingsWriterBase {
}
@Override
public void encodeTerm(DataOutput out, FieldInfo fieldInfo, BlockTermState _state, boolean absolute) throws IOException {
public void encodeTerm(long[] longs, DataOutput out, FieldInfo fieldInfo, BlockTermState _state, boolean absolute) throws IOException {
IntBlockTermState state = (IntBlockTermState)_state;
if (absolute) {
lastState = emptyState;
}
out.writeVLong(state.docStartFP - lastState.docStartFP);
longs[0] = state.docStartFP - lastState.docStartFP;
if (writePositions) {
out.writeVLong(state.posStartFP - lastState.posStartFP);
longs[1] = state.posStartFP - lastState.posStartFP;
if (writePayloads || writeOffsets) {
out.writeVLong(state.payStartFP - lastState.payStartFP);
longs[2] = state.payStartFP - lastState.payStartFP;
}
}
if (state.singletonDocID != -1) {

View File

@ -50,7 +50,7 @@ final class IDVersionPostingsReader extends PostingsReaderBase {
}
@Override
public void decodeTerm(DataInput in, FieldInfo fieldInfo, BlockTermState _termState, boolean absolute)
public void decodeTerm(long[] longs, DataInput in, FieldInfo fieldInfo, BlockTermState _termState, boolean absolute)
throws IOException {
final IDVersionTermState termState = (IDVersionTermState) _termState;
termState.docID = in.readVInt();

View File

@ -46,6 +46,7 @@ final class IDVersionPostingsWriter extends PushPostingsWriterBase {
private long lastVersion;
private final Bits liveDocs;
private String segment;
public IDVersionPostingsWriter(Bits liveDocs) {
this.liveDocs = liveDocs;
@ -59,10 +60,11 @@ final class IDVersionPostingsWriter extends PushPostingsWriterBase {
@Override
public void init(IndexOutput termsOut, SegmentWriteState state) throws IOException {
CodecUtil.writeIndexHeader(termsOut, TERMS_CODEC, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
segment = state.segmentInfo.name;
}
@Override
public void setField(FieldInfo fieldInfo) {
public int setField(FieldInfo fieldInfo) {
super.setField(fieldInfo);
if (fieldInfo.getIndexOptions() != IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
throw new IllegalArgumentException("field must be index using IndexOptions.DOCS_AND_FREQS_AND_POSITIONS");
@ -73,6 +75,7 @@ final class IDVersionPostingsWriter extends PushPostingsWriterBase {
throw new IllegalArgumentException("field cannot index term vectors: CheckIndex will report this as index corruption");
}
lastState = emptyState;
return 0;
}
@Override
@ -151,7 +154,7 @@ final class IDVersionPostingsWriter extends PushPostingsWriterBase {
private long lastEncodedVersion;
@Override
public void encodeTerm(DataOutput out, FieldInfo fieldInfo, BlockTermState _state, boolean absolute) throws IOException {
public void encodeTerm(long[] longs, DataOutput out, FieldInfo fieldInfo, BlockTermState _state, boolean absolute) throws IOException {
IDVersionTermState state = (IDVersionTermState) _state;
out.writeVInt(state.docID);
if (absolute) {

View File

@ -83,7 +83,9 @@ final class IDVersionSegmentTermsEnumFrame {
final BlockTermState state;
// metadata
// metadata buffer, holding monotonic values
public long[] longs;
// metadata buffer, holding general values
public byte[] bytes;
ByteArrayDataInput bytesReader;
@ -94,6 +96,7 @@ final class IDVersionSegmentTermsEnumFrame {
this.ord = ord;
this.state = ste.fr.parent.postingsReader.newTermState();
this.state.totalTermFreq = -1;
this.longs = new long[ste.fr.longsSize];
}
public void setFloorData(ByteArrayDataInput in, BytesRef source) {
@ -393,8 +396,11 @@ final class IDVersionSegmentTermsEnumFrame {
state.docFreq = 1;
state.totalTermFreq = 1;
//if (DEBUG) System.out.println(" dF=" + state.docFreq);
// metadata
ste.fr.parent.postingsReader.decodeTerm(bytesReader, ste.fr.fieldInfo, state, absolute);
// metadata
for (int i = 0; i < ste.fr.longsSize; i++) {
longs[i] = bytesReader.readVLong();
}
ste.fr.parent.postingsReader.decodeTerm(longs, bytesReader, ste.fr.fieldInfo, state, absolute);
metaDataUpto++;
absolute = false;

View File

@ -127,6 +127,7 @@ public final class VersionBlockTreeTermsReader extends FieldsProducer {
final long sumDocFreq = numTerms;
assert numTerms <= Integer.MAX_VALUE;
final int docCount = (int) numTerms;
final int longsSize = in.readVInt();
BytesRef minTerm = readBytesRef(in);
BytesRef maxTerm = readBytesRef(in);
@ -142,7 +143,7 @@ public final class VersionBlockTreeTermsReader extends FieldsProducer {
final long indexStartFP = indexIn.readVLong();
VersionFieldReader previous = fields.put(fieldInfo.name,
new VersionFieldReader(this, fieldInfo, numTerms, rootCode, sumTotalTermFreq, sumDocFreq, docCount,
indexStartFP, indexIn, minTerm, maxTerm));
indexStartFP, longsSize, indexIn, minTerm, maxTerm));
if (previous != null) {
throw new CorruptIndexException("duplicate field: " + fieldInfo.name, in);
}

View File

@ -143,10 +143,11 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
public final Pair<BytesRef,Long> rootCode;
public final long numTerms;
public final long indexStartFP;
private final int longsSize;
public final BytesRef minTerm;
public final BytesRef maxTerm;
public FieldMetaData(FieldInfo fieldInfo, Pair<BytesRef,Long> rootCode, long numTerms, long indexStartFP,
public FieldMetaData(FieldInfo fieldInfo, Pair<BytesRef,Long> rootCode, long numTerms, long indexStartFP, int longsSize,
BytesRef minTerm, BytesRef maxTerm) {
assert numTerms > 0;
this.fieldInfo = fieldInfo;
@ -154,6 +155,7 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
this.rootCode = rootCode;
this.indexStartFP = indexStartFP;
this.numTerms = numTerms;
this.longsSize = longsSize;
this.minTerm = minTerm;
this.maxTerm = maxTerm;
}
@ -401,6 +403,7 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
class TermsWriter {
private final FieldInfo fieldInfo;
private final int longsSize;
private long numTerms;
final FixedBitSet docsSeen;
long indexStartFP;
@ -413,6 +416,8 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
private final BytesRefBuilder lastTerm = new BytesRefBuilder();
private int[] prefixStarts = new int[8];
private final long[] longs;
// Pending stack of terms and blocks. As terms arrive (in sorted order)
// we append to this stack, and once the top of the stack has enough
// terms starting with a common prefix, we write a new block with
@ -600,7 +605,13 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
assert floorLeadLabel == -1 || (term.termBytes[prefixLength] & 0xff) >= floorLeadLabel;
// Write term meta data
postingsWriter.encodeTerm(metaWriter, fieldInfo, state, absolute);
postingsWriter.encodeTerm(longs, bytesWriter, fieldInfo, state, absolute);
for (int pos = 0; pos < longsSize; pos++) {
assert longs[pos] >= 0;
metaWriter.writeVLong(longs[pos]);
}
bytesWriter.copyTo(metaWriter);
bytesWriter.reset();
absolute = false;
}
} else {
@ -637,7 +648,13 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
// separate anymore:
// Write term meta data
postingsWriter.encodeTerm(metaWriter, fieldInfo, state, absolute);
postingsWriter.encodeTerm(longs, bytesWriter, fieldInfo, state, absolute);
for (int pos = 0; pos < longsSize; pos++) {
assert longs[pos] >= 0;
metaWriter.writeVLong(longs[pos]);
}
bytesWriter.copyTo(metaWriter);
bytesWriter.reset();
absolute = false;
} else {
PendingBlock block = (PendingBlock) ent;
@ -703,7 +720,8 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
this.fieldInfo = fieldInfo;
docsSeen = new FixedBitSet(maxDoc);
postingsWriter.setField(fieldInfo);
this.longsSize = postingsWriter.setField(fieldInfo);
this.longs = new long[longsSize];
}
/** Writes one term's worth of postings. */
@ -800,6 +818,7 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
((PendingBlock) pending.get(0)).index.getEmptyOutput(),
numTerms,
indexStartFP,
longsSize,
minTerm, maxTerm));
} else {
// cannot assert this: we skip deleted docIDs in the postings:
@ -809,6 +828,7 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
private final ByteBuffersDataOutput suffixWriter = ByteBuffersDataOutput.newResettableInstance();
private final ByteBuffersDataOutput metaWriter = ByteBuffersDataOutput.newResettableInstance();
private final ByteBuffersDataOutput bytesWriter = ByteBuffersDataOutput.newResettableInstance();
}
private boolean closed;
@ -836,6 +856,7 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
out.writeVInt(field.rootCode.output1.length);
out.writeBytes(field.rootCode.output1.bytes, field.rootCode.output1.offset, field.rootCode.output1.length);
out.writeVLong(field.rootCode.output2);
out.writeVInt(field.longsSize);
indexOut.writeVLong(field.indexStartFP);
writeBytesRef(out, field.minTerm);
writeBytesRef(out, field.maxTerm);

View File

@ -45,13 +45,14 @@ final class VersionFieldReader extends Terms implements Accountable {
final Pair<BytesRef,Long> rootCode;
final BytesRef minTerm;
final BytesRef maxTerm;
final int longsSize;
final VersionBlockTreeTermsReader parent;
final FST<Pair<BytesRef,Long>> index;
//private boolean DEBUG;
VersionFieldReader(VersionBlockTreeTermsReader parent, FieldInfo fieldInfo, long numTerms, Pair<BytesRef,Long> rootCode, long sumTotalTermFreq, long sumDocFreq, int docCount,
long indexStartFP, IndexInput indexIn, BytesRef minTerm, BytesRef maxTerm) throws IOException {
long indexStartFP, int longsSize, IndexInput indexIn, BytesRef minTerm, BytesRef maxTerm) throws IOException {
assert numTerms > 0;
this.fieldInfo = fieldInfo;
//DEBUG = BlockTreeTermsReader.DEBUG && fieldInfo.name.equals("id");
@ -62,6 +63,7 @@ final class VersionFieldReader extends Terms implements Accountable {
this.docCount = docCount;
this.indexStartFP = indexStartFP;
this.rootCode = rootCode;
this.longsSize = longsSize;
this.minTerm = minTerm;
this.maxTerm = maxTerm;
// if (DEBUG) {

View File

@ -41,6 +41,10 @@ import org.apache.lucene.codecs.blocktreeords.OrdsBlockTreeTermsReader;
import org.apache.lucene.codecs.blocktreeords.OrdsBlockTreeTermsWriter;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsReader;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsWriter;
import org.apache.lucene.codecs.memory.FSTOrdTermsReader;
import org.apache.lucene.codecs.memory.FSTOrdTermsWriter;
import org.apache.lucene.codecs.memory.FSTTermsReader;
import org.apache.lucene.codecs.memory.FSTTermsWriter;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentReadState;
@ -118,9 +122,29 @@ public final class MockRandomPostingsFormat extends PostingsFormat {
PostingsWriterBase postingsWriter = new Lucene84PostingsWriter(state);
final FieldsConsumer fields;
final int t1 = random.nextInt(3);
final int t1 = random.nextInt(5);
if (t1 == 0) {
if (t1 == 0) {
boolean success = false;
try {
fields = new FSTTermsWriter(state, postingsWriter);
success = true;
} finally {
if (!success) {
postingsWriter.close();
}
}
} else if (t1 == 1) {
boolean success = false;
try {
fields = new FSTOrdTermsWriter(state, postingsWriter);
success = true;
} finally {
if (!success) {
postingsWriter.close();
}
}
} else if (t1 == 2) {
// Use BlockTree terms dict
if (LuceneTestCase.VERBOSE) {
@ -141,7 +165,7 @@ public final class MockRandomPostingsFormat extends PostingsFormat {
postingsWriter.close();
}
}
} else if (t1 == 1) {
} else if (t1 == 3) {
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: writing Block terms dict");
@ -211,7 +235,7 @@ public final class MockRandomPostingsFormat extends PostingsFormat {
}
}
}
} else if (t1 == 2) {
} else if (t1 == 4) {
// Use OrdsBlockTree terms dict
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: writing OrdsBlockTree");
@ -263,8 +287,28 @@ public final class MockRandomPostingsFormat extends PostingsFormat {
PostingsReaderBase postingsReader = new Lucene84PostingsReader(state);
final FieldsProducer fields;
final int t1 = random.nextInt(3);
final int t1 = random.nextInt(5);
if (t1 == 0) {
boolean success = false;
try {
fields = new FSTTermsReader(state, postingsReader);
success = true;
} finally {
if (!success) {
postingsReader.close();
}
}
} else if (t1 == 1) {
boolean success = false;
try {
fields = new FSTOrdTermsReader(state, postingsReader);
success = true;
} finally {
if (!success) {
postingsReader.close();
}
}
} else if (t1 == 2) {
// Use BlockTree terms dict
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: reading BlockTree terms dict");
@ -279,7 +323,7 @@ public final class MockRandomPostingsFormat extends PostingsFormat {
postingsReader.close();
}
}
} else if (t1 == 1) {
} else if (t1 == 3) {
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: reading Block terms dict");
@ -330,7 +374,7 @@ public final class MockRandomPostingsFormat extends PostingsFormat {
}
}
}
} else if (t1 == 2) {
} else if (t1 == 4) {
// Use OrdsBlockTree terms dict
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: reading OrdsBlockTree terms dict");

View File

@ -45,6 +45,8 @@ import org.apache.lucene.codecs.bloom.TestBloomFilteredLucenePostings;
import org.apache.lucene.codecs.lucene60.Lucene60PointsReader;
import org.apache.lucene.codecs.lucene60.Lucene60PointsWriter;
import org.apache.lucene.codecs.memory.DirectPostingsFormat;
import org.apache.lucene.codecs.memory.FSTOrdPostingsFormat;
import org.apache.lucene.codecs.memory.FSTPostingsFormat;
import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
import org.apache.lucene.index.PointValues.IntersectVisitor;
import org.apache.lucene.store.Directory;
@ -187,6 +189,8 @@ public class RandomCodec extends AssertingCodec {
add(avoidCodecs,
TestUtil.getDefaultPostingsFormat(minItemsPerBlock, maxItemsPerBlock, RandomPicks.randomFrom(random, BlockTreeTermsReader.FSTLoadMode.values())),
new FSTPostingsFormat(),
new FSTOrdPostingsFormat(),
new DirectPostingsFormat(LuceneTestCase.rarely(random) ? 1 : (LuceneTestCase.rarely(random) ? Integer.MAX_VALUE : maxItemsPerBlock),
LuceneTestCase.rarely(random) ? 1 : (LuceneTestCase.rarely(random) ? Integer.MAX_VALUE : lowFreqCutoff)),
//TODO as a PostingsFormat which wraps others, we should allow TestBloomFilteredLucenePostings to be constructed