LUCENE-5969: move old postings back compat to backward-codecs, cleanup PBF related stuff, add segment headers, etc

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene5969@1633196 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Robert Muir 2014-10-20 18:25:52 +00:00
parent c9e2d7bead
commit eee7825cef
81 changed files with 3800 additions and 615 deletions

View File

@ -1,48 +0,0 @@
package org.apache.lucene.codecs.lucene40;
/*
* 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.
*/
import java.io.IOException;
import org.apache.lucene.codecs.PostingsBaseFormat;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
/**
* PostingsReaderBase for 4.0 segments
* @deprecated Only for reading old 4.0 segments */
@Deprecated
final class Lucene40PostingsBaseFormat extends PostingsBaseFormat {
/** Sole constructor. */
Lucene40PostingsBaseFormat() {
super("Lucene40");
}
@Override
public PostingsReaderBase postingsReaderBase(SegmentReadState state) throws IOException {
return new Lucene40PostingsReader(state.directory, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
}
@Override
public PostingsWriterBase postingsWriterBase(SegmentWriteState state) throws IOException {
throw new UnsupportedOperationException("this codec can only be used for reading");
}
}

View File

@ -51,13 +51,7 @@ public class Lucene40PostingsFormat extends PostingsFormat {
boolean success = false;
try {
FieldsProducer ret = new BlockTreeTermsReader(
state.directory,
state.fieldInfos,
state.segmentInfo,
postings,
state.context,
state.segmentSuffix);
FieldsProducer ret = new BlockTreeTermsReader(postings, state);
success = true;
return ret;
} finally {

View File

@ -31,6 +31,7 @@ import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.TermState;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.Directory;
@ -104,7 +105,7 @@ final class Lucene40PostingsReader extends PostingsReaderBase {
}
@Override
public void init(IndexInput termsIn) throws IOException {
public void init(IndexInput termsIn, SegmentReadState state) throws IOException {
// Make sure we are talking to the matching past writer
CodecUtil.checkHeader(termsIn, TERMS_CODEC, VERSION_START, VERSION_CURRENT);

View File

@ -0,0 +1,247 @@
package org.apache.lucene.codecs.lucene41;
/*
* 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.
*/
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.packed.PackedInts.Decoder;
import org.apache.lucene.util.packed.PackedInts.FormatAndBits;
import org.apache.lucene.util.packed.PackedInts;
import static org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat.BLOCK_SIZE;
/**
* Lucene 4.1 postings format.
* @deprecated only for reading old 4.x segments
*/
@Deprecated
final class ForUtil {
/**
* Special number of bits per value used whenever all values to encode are equal.
*/
private static final int ALL_VALUES_EQUAL = 0;
/**
* Upper limit of the number of bytes that might be required to stored
* <code>BLOCK_SIZE</code> encoded values.
*/
static final int MAX_ENCODED_SIZE = BLOCK_SIZE * 4;
/**
* Upper limit of the number of values that might be decoded in a single call to
* {@link #readBlock(IndexInput, byte[], int[])}. Although values after
* <code>BLOCK_SIZE</code> are garbage, it is necessary to allocate value buffers
* whose size is >= MAX_DATA_SIZE to avoid {@link ArrayIndexOutOfBoundsException}s.
*/
static final int MAX_DATA_SIZE;
static {
int maxDataSize = 0;
for(int version=PackedInts.VERSION_START;version<=PackedInts.VERSION_CURRENT;version++) {
for (PackedInts.Format format : PackedInts.Format.values()) {
for (int bpv = 1; bpv <= 32; ++bpv) {
if (!format.isSupported(bpv)) {
continue;
}
final PackedInts.Decoder decoder = PackedInts.getDecoder(format, version, bpv);
final int iterations = computeIterations(decoder);
maxDataSize = Math.max(maxDataSize, iterations * decoder.byteValueCount());
}
}
}
MAX_DATA_SIZE = maxDataSize;
}
/**
* Compute the number of iterations required to decode <code>BLOCK_SIZE</code>
* values with the provided {@link Decoder}.
*/
private static int computeIterations(PackedInts.Decoder decoder) {
return (int) Math.ceil((float) BLOCK_SIZE / decoder.byteValueCount());
}
/**
* Compute the number of bytes required to encode a block of values that require
* <code>bitsPerValue</code> bits per value with format <code>format</code>.
*/
private static int encodedSize(PackedInts.Format format, int packedIntsVersion, int bitsPerValue) {
final long byteCount = format.byteCount(packedIntsVersion, BLOCK_SIZE, bitsPerValue);
assert byteCount >= 0 && byteCount <= Integer.MAX_VALUE : byteCount;
return (int) byteCount;
}
private final int[] encodedSizes;
private final PackedInts.Encoder[] encoders;
private final PackedInts.Decoder[] decoders;
private final int[] iterations;
/**
* Create a new {@link ForUtil} instance and save state into <code>out</code>.
*/
ForUtil(float acceptableOverheadRatio, DataOutput out) throws IOException {
out.writeVInt(PackedInts.VERSION_CURRENT);
encodedSizes = new int[33];
encoders = new PackedInts.Encoder[33];
decoders = new PackedInts.Decoder[33];
iterations = new int[33];
for (int bpv = 1; bpv <= 32; ++bpv) {
final FormatAndBits formatAndBits = PackedInts.fastestFormatAndBits(
BLOCK_SIZE, bpv, acceptableOverheadRatio);
assert formatAndBits.format.isSupported(formatAndBits.bitsPerValue);
assert formatAndBits.bitsPerValue <= 32;
encodedSizes[bpv] = encodedSize(formatAndBits.format, PackedInts.VERSION_CURRENT, formatAndBits.bitsPerValue);
encoders[bpv] = PackedInts.getEncoder(
formatAndBits.format, PackedInts.VERSION_CURRENT, formatAndBits.bitsPerValue);
decoders[bpv] = PackedInts.getDecoder(
formatAndBits.format, PackedInts.VERSION_CURRENT, formatAndBits.bitsPerValue);
iterations[bpv] = computeIterations(decoders[bpv]);
out.writeVInt(formatAndBits.format.getId() << 5 | (formatAndBits.bitsPerValue - 1));
}
}
/**
* Restore a {@link ForUtil} from a {@link DataInput}.
*/
ForUtil(DataInput in) throws IOException {
int packedIntsVersion = in.readVInt();
PackedInts.checkVersion(packedIntsVersion);
encodedSizes = new int[33];
encoders = new PackedInts.Encoder[33];
decoders = new PackedInts.Decoder[33];
iterations = new int[33];
for (int bpv = 1; bpv <= 32; ++bpv) {
final int code = in.readVInt();
final int formatId = code >>> 5;
final int bitsPerValue = (code & 31) + 1;
final PackedInts.Format format = PackedInts.Format.byId(formatId);
assert format.isSupported(bitsPerValue);
encodedSizes[bpv] = encodedSize(format, packedIntsVersion, bitsPerValue);
encoders[bpv] = PackedInts.getEncoder(
format, packedIntsVersion, bitsPerValue);
decoders[bpv] = PackedInts.getDecoder(
format, packedIntsVersion, bitsPerValue);
iterations[bpv] = computeIterations(decoders[bpv]);
}
}
/**
* Write a block of data (<code>For</code> format).
*
* @param data the data to write
* @param encoded a buffer to use to encode data
* @param out the destination output
* @throws IOException If there is a low-level I/O error
*/
void writeBlock(int[] data, byte[] encoded, IndexOutput out) throws IOException {
if (isAllEqual(data)) {
out.writeByte((byte) ALL_VALUES_EQUAL);
out.writeVInt(data[0]);
return;
}
final int numBits = bitsRequired(data);
assert numBits > 0 && numBits <= 32 : numBits;
final PackedInts.Encoder encoder = encoders[numBits];
final int iters = iterations[numBits];
assert iters * encoder.byteValueCount() >= BLOCK_SIZE;
final int encodedSize = encodedSizes[numBits];
assert iters * encoder.byteBlockCount() >= encodedSize;
out.writeByte((byte) numBits);
encoder.encode(data, 0, encoded, 0, iters);
out.writeBytes(encoded, encodedSize);
}
/**
* Read the next block of data (<code>For</code> format).
*
* @param in the input to use to read data
* @param encoded a buffer that can be used to store encoded data
* @param decoded where to write decoded data
* @throws IOException If there is a low-level I/O error
*/
void readBlock(IndexInput in, byte[] encoded, int[] decoded) throws IOException {
final int numBits = in.readByte();
assert numBits <= 32 : numBits;
if (numBits == ALL_VALUES_EQUAL) {
final int value = in.readVInt();
Arrays.fill(decoded, 0, BLOCK_SIZE, value);
return;
}
final int encodedSize = encodedSizes[numBits];
in.readBytes(encoded, 0, encodedSize);
final PackedInts.Decoder decoder = decoders[numBits];
final int iters = iterations[numBits];
assert iters * decoder.byteValueCount() >= BLOCK_SIZE;
decoder.decode(encoded, 0, decoded, 0, iters);
}
/**
* Skip the next block of data.
*
* @param in the input where to read data
* @throws IOException If there is a low-level I/O error
*/
void skipBlock(IndexInput in) throws IOException {
final int numBits = in.readByte();
if (numBits == ALL_VALUES_EQUAL) {
in.readVInt();
return;
}
assert numBits > 0 && numBits <= 32 : numBits;
final int encodedSize = encodedSizes[numBits];
in.seek(in.getFilePointer() + encodedSize);
}
private static boolean isAllEqual(final int[] data) {
final int v = data[0];
for (int i = 1; i < BLOCK_SIZE; ++i) {
if (data[i] != v) {
return false;
}
}
return true;
}
/**
* Compute the number of bits required to serialize any of the longs in
* <code>data</code>.
*/
private static int bitsRequired(final int[] data) {
long or = 0;
for (int i = 0; i < BLOCK_SIZE; ++i) {
assert data[i] >= 0;
or |= data[i];
}
return PackedInts.bitsRequired(or);
}
}

View File

@ -0,0 +1,45 @@
package org.apache.lucene.codecs.lucene41;
import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.index.TermState;
/**
* term state for Lucene 4.1 postings format
* @deprecated only for reading old 4.x segments
*/
@Deprecated
final class IntBlockTermState extends BlockTermState {
long docStartFP = 0;
long posStartFP = 0;
long payStartFP = 0;
long skipOffset = -1;
long lastPosBlockOffset = -1;
// docid when there is a single pulsed posting, otherwise -1
// freq is always implicitly totalTermFreq in this case.
int singletonDocID = -1;
@Override
public IntBlockTermState clone() {
IntBlockTermState other = new IntBlockTermState();
other.copyFrom(this);
return other;
}
@Override
public void copyFrom(TermState _other) {
super.copyFrom(_other);
IntBlockTermState other = (IntBlockTermState) _other;
docStartFP = other.docStartFP;
posStartFP = other.posStartFP;
payStartFP = other.payStartFP;
lastPosBlockOffset = other.lastPosBlockOffset;
skipOffset = other.skipOffset;
singletonDocID = other.singletonDocID;
}
@Override
public String toString() {
return super.toString() + " docStartFP=" + docStartFP + " posStartFP=" + posStartFP + " payStartFP=" + payStartFP + " lastPosBlockOffset=" + lastPosBlockOffset + " singletonDocID=" + singletonDocID;
}
}

View File

@ -0,0 +1,113 @@
package org.apache.lucene.codecs.lucene41;
/*
* 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.
*/
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.blocktree.BlockTreeTermsReader;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.IOUtils;
/**
* Lucene 4.1 postings format.
* @deprecated only for reading old 4.x segments
*/
@Deprecated
public class Lucene41PostingsFormat extends PostingsFormat {
/**
* Filename extension for document number, frequencies, and skip data.
* See chapter: <a href="#Frequencies">Frequencies and Skip Data</a>
*/
public static final String DOC_EXTENSION = "doc";
/**
* Filename extension for positions.
* See chapter: <a href="#Positions">Positions</a>
*/
public static final String POS_EXTENSION = "pos";
/**
* Filename extension for payloads and offsets.
* See chapter: <a href="#Payloads">Payloads and Offsets</a>
*/
public static final String PAY_EXTENSION = "pay";
/**
* Expert: The maximum number of skip levels. Smaller values result in
* slightly smaller indexes, but slower skipping in big posting lists.
*/
static final int maxSkipLevels = 10;
final static String TERMS_CODEC = "Lucene41PostingsWriterTerms";
final static String DOC_CODEC = "Lucene41PostingsWriterDoc";
final static String POS_CODEC = "Lucene41PostingsWriterPos";
final static String PAY_CODEC = "Lucene41PostingsWriterPay";
// Increment version to change it
final static int VERSION_START = 0;
final static int VERSION_META_ARRAY = 1;
final static int VERSION_CHECKSUM = 2;
final static int VERSION_CURRENT = VERSION_CHECKSUM;
/**
* Fixed packed block size, number of integers encoded in
* a single packed block.
*/
// NOTE: must be multiple of 64 because of PackedInts long-aligned encoding/decoding
public final static int BLOCK_SIZE = 128;
/** Creates {@code Lucene41PostingsFormat} with default
* settings. */
public Lucene41PostingsFormat() {
super("Lucene41");
}
@Override
public String toString() {
return getName() + "(blocksize=" + BLOCK_SIZE + ")";
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
throw new UnsupportedOperationException("this codec can only be used for reading");
}
@Override
public final FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postingsReader = new Lucene41PostingsReader(state.directory,
state.fieldInfos,
state.segmentInfo,
state.context,
state.segmentSuffix);
boolean success = false;
try {
FieldsProducer ret = new BlockTreeTermsReader(postingsReader, state);
success = true;
return ret;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(postingsReader);
}
}
}
}

View File

@ -20,7 +20,6 @@ package org.apache.lucene.codecs.lucene41;
import static org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat.BLOCK_SIZE;
import static org.apache.lucene.codecs.lucene41.ForUtil.MAX_DATA_SIZE;
import static org.apache.lucene.codecs.lucene41.ForUtil.MAX_ENCODED_SIZE;
import static org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter.IntBlockTermState;
import java.io.IOException;
import java.util.Arrays;
@ -32,6 +31,7 @@ import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
@ -48,12 +48,10 @@ import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.RamUsageEstimator;
/**
* Concrete class that reads docId(maybe frq,pos,offset,payloads) list
* with postings format.
*
* @see Lucene41SkipReader for details
* @lucene.experimental
* Lucene 4.1 postings format.
* @deprecated only for reading old 4.x segments
*/
@Deprecated
public final class Lucene41PostingsReader extends PostingsReaderBase {
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(Lucene41PostingsReader.class);
@ -77,12 +75,12 @@ public final class Lucene41PostingsReader extends PostingsReaderBase {
docIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene41PostingsFormat.DOC_EXTENSION),
ioContext);
version = CodecUtil.checkHeader(docIn,
Lucene41PostingsWriter.DOC_CODEC,
Lucene41PostingsWriter.VERSION_START,
Lucene41PostingsWriter.VERSION_CURRENT);
Lucene41PostingsFormat.DOC_CODEC,
Lucene41PostingsFormat.VERSION_START,
Lucene41PostingsFormat.VERSION_CURRENT);
forUtil = new ForUtil(docIn);
if (version >= Lucene41PostingsWriter.VERSION_CHECKSUM) {
if (version >= Lucene41PostingsFormat.VERSION_CHECKSUM) {
// NOTE: data file is too costly to verify checksum against all the bytes on open,
// but for now we at least verify proper structure of the checksum footer: which looks
// for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption
@ -93,9 +91,9 @@ public final class Lucene41PostingsReader extends PostingsReaderBase {
if (fieldInfos.hasProx()) {
posIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene41PostingsFormat.POS_EXTENSION),
ioContext);
CodecUtil.checkHeader(posIn, Lucene41PostingsWriter.POS_CODEC, version, version);
CodecUtil.checkHeader(posIn, Lucene41PostingsFormat.POS_CODEC, version, version);
if (version >= Lucene41PostingsWriter.VERSION_CHECKSUM) {
if (version >= Lucene41PostingsFormat.VERSION_CHECKSUM) {
// NOTE: data file is too costly to verify checksum against all the bytes on open,
// but for now we at least verify proper structure of the checksum footer: which looks
// for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption
@ -106,9 +104,9 @@ public final class Lucene41PostingsReader extends PostingsReaderBase {
if (fieldInfos.hasPayloads() || fieldInfos.hasOffsets()) {
payIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene41PostingsFormat.PAY_EXTENSION),
ioContext);
CodecUtil.checkHeader(payIn, Lucene41PostingsWriter.PAY_CODEC, version, version);
CodecUtil.checkHeader(payIn, Lucene41PostingsFormat.PAY_CODEC, version, version);
if (version >= Lucene41PostingsWriter.VERSION_CHECKSUM) {
if (version >= Lucene41PostingsFormat.VERSION_CHECKSUM) {
// NOTE: data file is too costly to verify checksum against all the bytes on open,
// but for now we at least verify proper structure of the checksum footer: which looks
// for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption
@ -130,12 +128,12 @@ public final class Lucene41PostingsReader extends PostingsReaderBase {
}
@Override
public void init(IndexInput termsIn) throws IOException {
public void init(IndexInput termsIn, SegmentReadState state) throws IOException {
// Make sure we are talking to the matching postings writer
CodecUtil.checkHeader(termsIn,
Lucene41PostingsWriter.TERMS_CODEC,
Lucene41PostingsWriter.VERSION_START,
Lucene41PostingsWriter.VERSION_CURRENT);
Lucene41PostingsFormat.TERMS_CODEC,
Lucene41PostingsFormat.VERSION_START,
Lucene41PostingsFormat.VERSION_CURRENT);
final int indexBlockSize = termsIn.readVInt();
if (indexBlockSize != BLOCK_SIZE) {
throw new IllegalStateException("index-time BLOCK_SIZE (" + indexBlockSize + ") != read-time BLOCK_SIZE (" + BLOCK_SIZE + ")");
@ -187,7 +185,7 @@ public final class Lucene41PostingsReader extends PostingsReaderBase {
termState.posStartFP = 0;
termState.payStartFP = 0;
}
if (version < Lucene41PostingsWriter.VERSION_META_ARRAY) { // backward compatibility
if (version < Lucene41PostingsFormat.VERSION_META_ARRAY) { // backward compatibility
_decodeTerm(in, fieldInfo, termState);
return;
}
@ -488,7 +486,7 @@ public final class Lucene41PostingsReader extends PostingsReaderBase {
if (skipper == null) {
// Lazy init: first time this enum has ever been used for skipping
skipper = new Lucene41SkipReader(docIn.clone(),
Lucene41PostingsWriter.maxSkipLevels,
Lucene41PostingsFormat.maxSkipLevels,
BLOCK_SIZE,
indexHasPos,
indexHasOffsets,
@ -821,7 +819,7 @@ public final class Lucene41PostingsReader extends PostingsReaderBase {
// System.out.println(" create skipper");
// }
skipper = new Lucene41SkipReader(docIn.clone(),
Lucene41PostingsWriter.maxSkipLevels,
Lucene41PostingsFormat.maxSkipLevels,
BLOCK_SIZE,
true,
indexHasOffsets,
@ -1347,7 +1345,7 @@ public final class Lucene41PostingsReader extends PostingsReaderBase {
// System.out.println(" create skipper");
// }
skipper = new Lucene41SkipReader(docIn.clone(),
Lucene41PostingsWriter.maxSkipLevels,
Lucene41PostingsFormat.maxSkipLevels,
BLOCK_SIZE,
true,
indexHasOffsets,
@ -1590,7 +1588,7 @@ public final class Lucene41PostingsReader extends PostingsReaderBase {
@Override
public void checkIntegrity() throws IOException {
if (version >= Lucene41PostingsWriter.VERSION_CHECKSUM) {
if (version >= Lucene41PostingsFormat.VERSION_CHECKSUM) {
if (docIn != null) {
CodecUtil.checksumEntireFile(docIn);
}

View File

@ -24,32 +24,10 @@ import org.apache.lucene.codecs.MultiLevelSkipListReader;
import org.apache.lucene.store.IndexInput;
/**
* Implements the skip list reader for block postings format
* that stores positions and payloads.
*
* Although this skipper uses MultiLevelSkipListReader as an interface,
* its definition of skip position will be a little different.
*
* For example, when skipInterval = blockSize = 3, df = 2*skipInterval = 6,
*
* 0 1 2 3 4 5
* d d d d d d (posting list)
* ^ ^ (skip point in MultiLeveSkipWriter)
* ^ (skip point in Lucene41SkipWriter)
*
* In this case, MultiLevelSkipListReader will use the last document as a skip point,
* while Lucene41SkipReader should assume no skip point will comes.
*
* If we use the interface directly in Lucene41SkipReader, it may silly try to read
* another skip data after the only skip point is loaded.
*
* To illustrate this, we can call skipTo(d[5]), since skip point d[3] has smaller docId,
* and numSkipped+blockSize== df, the MultiLevelSkipListReader will assume the skip list
* isn't exhausted yet, and try to load a non-existed skip point
*
* Therefore, we'll trim df before passing it to the interface. see trim(int)
*
* Lucene 4.1 skiplist format.
* @deprecated only for reading old 4.x segments
*/
@Deprecated
final class Lucene41SkipReader extends MultiLevelSkipListReader {
// private boolean DEBUG = Lucene41PostingsReader.DEBUG;
private final int blockSize;

View File

@ -14,3 +14,4 @@
# limitations under the License.
org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat
org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat

View File

@ -129,7 +129,7 @@ final class Lucene40PostingsWriter extends PushPostingsWriterBase {
}
@Override
public void init(IndexOutput termsOut) throws IOException {
public void init(IndexOutput termsOut, SegmentWriteState state) throws IOException {
CodecUtil.writeHeader(termsOut, Lucene40PostingsReader.TERMS_CODEC, Lucene40PostingsReader.VERSION_CURRENT);
termsOut.writeInt(skipInterval); // write skipInterval
termsOut.writeInt(maxSkipLevels); // write maxSkipLevels

View File

@ -26,7 +26,6 @@ import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.TermState;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.ArrayUtil;
@ -38,35 +37,13 @@ import static org.apache.lucene.codecs.lucene41.ForUtil.MAX_DATA_SIZE;
import static org.apache.lucene.codecs.lucene41.ForUtil.MAX_ENCODED_SIZE;
import static org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat.BLOCK_SIZE;
/**
* Concrete class that writes docId(maybe frq,pos,offset,payloads) list
* with postings format.
*
* Postings list for each term will be stored separately.
*
* @see Lucene41SkipWriter for details about skipping setting and postings layout.
* @lucene.experimental
* Writes 4.1 postings for testing
* @deprecated for test purposes only
*/
@Deprecated
public final class Lucene41PostingsWriter extends PushPostingsWriterBase {
/**
* Expert: The maximum number of skip levels. Smaller values result in
* slightly smaller indexes, but slower skipping in big posting lists.
*/
static final int maxSkipLevels = 10;
final static String TERMS_CODEC = "Lucene41PostingsWriterTerms";
final static String DOC_CODEC = "Lucene41PostingsWriterDoc";
final static String POS_CODEC = "Lucene41PostingsWriterPos";
final static String PAY_CODEC = "Lucene41PostingsWriterPay";
// Increment version to change it
final static int VERSION_START = 0;
final static int VERSION_META_ARRAY = 1;
final static int VERSION_CHECKSUM = 2;
final static int VERSION_CURRENT = VERSION_CHECKSUM;
IndexOutput docOut;
IndexOutput posOut;
IndexOutput payOut;
@ -119,13 +96,13 @@ public final class Lucene41PostingsWriter extends PushPostingsWriterBase {
IndexOutput payOut = null;
boolean success = false;
try {
CodecUtil.writeHeader(docOut, DOC_CODEC, VERSION_CURRENT);
CodecUtil.writeHeader(docOut, Lucene41PostingsFormat.DOC_CODEC, Lucene41PostingsFormat.VERSION_CURRENT);
forUtil = new ForUtil(acceptableOverheadRatio, docOut);
if (state.fieldInfos.hasProx()) {
posDeltaBuffer = new int[MAX_DATA_SIZE];
posOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene41PostingsFormat.POS_EXTENSION),
state.context);
CodecUtil.writeHeader(posOut, POS_CODEC, VERSION_CURRENT);
CodecUtil.writeHeader(posOut, Lucene41PostingsFormat.POS_CODEC, Lucene41PostingsFormat.VERSION_CURRENT);
if (state.fieldInfos.hasPayloads()) {
payloadBytes = new byte[128];
@ -146,7 +123,7 @@ public final class Lucene41PostingsWriter extends PushPostingsWriterBase {
if (state.fieldInfos.hasPayloads() || state.fieldInfos.hasOffsets()) {
payOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene41PostingsFormat.PAY_EXTENSION),
state.context);
CodecUtil.writeHeader(payOut, PAY_CODEC, VERSION_CURRENT);
CodecUtil.writeHeader(payOut, Lucene41PostingsFormat.PAY_CODEC, Lucene41PostingsFormat.VERSION_CURRENT);
}
} else {
posDeltaBuffer = null;
@ -168,7 +145,7 @@ public final class Lucene41PostingsWriter extends PushPostingsWriterBase {
freqBuffer = new int[MAX_DATA_SIZE];
// TODO: should we try skipping every 2/4 blocks...?
skipWriter = new Lucene41SkipWriter(maxSkipLevels,
skipWriter = new Lucene41SkipWriter(Lucene41PostingsFormat.maxSkipLevels,
BLOCK_SIZE,
state.segmentInfo.getDocCount(),
docOut,
@ -183,50 +160,14 @@ public final class Lucene41PostingsWriter extends PushPostingsWriterBase {
this(state, PackedInts.COMPACT);
}
final static class IntBlockTermState extends BlockTermState {
long docStartFP = 0;
long posStartFP = 0;
long payStartFP = 0;
long skipOffset = -1;
long lastPosBlockOffset = -1;
// docid when there is a single pulsed posting, otherwise -1
// freq is always implicitly totalTermFreq in this case.
int singletonDocID = -1;
@Override
public IntBlockTermState clone() {
IntBlockTermState other = new IntBlockTermState();
other.copyFrom(this);
return other;
}
@Override
public void copyFrom(TermState _other) {
super.copyFrom(_other);
IntBlockTermState other = (IntBlockTermState) _other;
docStartFP = other.docStartFP;
posStartFP = other.posStartFP;
payStartFP = other.payStartFP;
lastPosBlockOffset = other.lastPosBlockOffset;
skipOffset = other.skipOffset;
singletonDocID = other.singletonDocID;
}
@Override
public String toString() {
return super.toString() + " docStartFP=" + docStartFP + " posStartFP=" + posStartFP + " payStartFP=" + payStartFP + " lastPosBlockOffset=" + lastPosBlockOffset + " singletonDocID=" + singletonDocID;
}
}
@Override
public IntBlockTermState newTermState() {
return new IntBlockTermState();
}
@Override
public void init(IndexOutput termsOut) throws IOException {
CodecUtil.writeHeader(termsOut, TERMS_CODEC, VERSION_CURRENT);
public void init(IndexOutput termsOut, SegmentWriteState state) throws IOException {
CodecUtil.writeHeader(termsOut, Lucene41PostingsFormat.TERMS_CODEC, Lucene41PostingsFormat.VERSION_CURRENT);
termsOut.writeVInt(BLOCK_SIZE);
}

View File

@ -3,6 +3,7 @@ package org.apache.lucene.codecs.lucene41;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
@ -40,6 +41,12 @@ public final class Lucene41RWCodec extends Lucene41Codec {
private final DocValuesFormat docValues = new Lucene40RWDocValuesFormat();
private final NormsFormat norms = new Lucene40RWNormsFormat();
private final TermVectorsFormat vectors = new Lucene40RWTermVectorsFormat();
private final PostingsFormat postings = new Lucene41RWPostingsFormat();
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return postings;
}
@Override
public FieldInfosFormat fieldInfosFormat() {

View File

@ -0,0 +1,56 @@
package org.apache.lucene.codecs.lucene41;
/*
* 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.
*/
import java.io.IOException;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.IOUtils;
/**
* Read-write version of 4.1 postings format for testing
* @deprecated for test purposes only
*/
@Deprecated
public class Lucene41RWPostingsFormat extends Lucene41PostingsFormat {
static final int MIN_BLOCK_SIZE = 25;
static final int MAX_BLOCK_SIZE = 48;
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase postingsWriter = new Lucene41PostingsWriter(state);
boolean success = false;
try {
FieldsConsumer ret = new BlockTreeTermsWriter(state,
postingsWriter,
MIN_BLOCK_SIZE,
MAX_BLOCK_SIZE);
success = true;
return ret;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(postingsWriter);
}
}
}
}

View File

@ -24,25 +24,10 @@ import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.codecs.MultiLevelSkipListWriter;
/**
* Write skip lists with multiple levels, and support skip within block ints.
*
* Assume that docFreq = 28, skipInterval = blockSize = 12
*
* | block#0 | | block#1 | |vInts|
* d d d d d d d d d d d d d d d d d d d d d d d d d d d d (posting list)
* ^ ^ (level 0 skip point)
*
* Note that skipWriter will ignore first document in block#0, since
* it is useless as a skip point. Also, we'll never skip into the vInts
* block, only record skip data at the start its start point(if it exist).
*
* For each skip point, we will record:
* 1. docID in former position, i.e. for position 12, record docID[11], etc.
* 2. its related file points(position, payload),
* 3. related numbers or uptos(position, payload).
* 4. start offset.
*
* Writes 4.1 skiplists for testing
* @deprecated for test purposes only
*/
@Deprecated
final class Lucene41SkipWriter extends MultiLevelSkipListWriter {
// private boolean DEBUG = Lucene41PostingsReader.DEBUG;

View File

@ -0,0 +1,94 @@
package org.apache.lucene.codecs.lucene41;
/*
* 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.
*/
import static org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat.BLOCK_SIZE;
import static org.apache.lucene.codecs.lucene41.ForUtil.MAX_DATA_SIZE;
import static org.apache.lucene.codecs.lucene41.ForUtil.MAX_ENCODED_SIZE;
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.packed.PackedInts;
import com.carrotsearch.randomizedtesting.generators.RandomInts;
public class TestLucene41ForUtil extends LuceneTestCase {
public void testEncodeDecode() throws IOException {
final int iterations = RandomInts.randomIntBetween(random(), 1, 1000);
final float acceptableOverheadRatio = random().nextFloat();
final int[] values = new int[(iterations - 1) * BLOCK_SIZE + ForUtil.MAX_DATA_SIZE];
for (int i = 0; i < iterations; ++i) {
final int bpv = random().nextInt(32);
if (bpv == 0) {
final int value = RandomInts.randomIntBetween(random(), 0, Integer.MAX_VALUE);
for (int j = 0; j < BLOCK_SIZE; ++j) {
values[i * BLOCK_SIZE + j] = value;
}
} else {
for (int j = 0; j < BLOCK_SIZE; ++j) {
values[i * BLOCK_SIZE + j] = RandomInts.randomIntBetween(random(),
0, (int) PackedInts.maxValue(bpv));
}
}
}
final Directory d = new RAMDirectory();
final long endPointer;
{
// encode
IndexOutput out = d.createOutput("test.bin", IOContext.DEFAULT);
final ForUtil forUtil = new ForUtil(acceptableOverheadRatio, out);
for (int i = 0; i < iterations; ++i) {
forUtil.writeBlock(
Arrays.copyOfRange(values, i * BLOCK_SIZE, values.length),
new byte[MAX_ENCODED_SIZE], out);
}
endPointer = out.getFilePointer();
out.close();
}
{
// decode
IndexInput in = d.openInput("test.bin", IOContext.READONCE);
final ForUtil forUtil = new ForUtil(in);
for (int i = 0; i < iterations; ++i) {
if (random().nextBoolean()) {
forUtil.skipBlock(in);
continue;
}
final int[] restored = new int[MAX_DATA_SIZE];
forUtil.readBlock(in, new byte[MAX_ENCODED_SIZE], restored);
assertArrayEquals(Arrays.copyOfRange(values, i * BLOCK_SIZE, (i + 1) * BLOCK_SIZE),
Arrays.copyOf(restored, BLOCK_SIZE));
}
assertEquals(endPointer, in.getFilePointer());
in.close();
}
}
}

View File

@ -0,0 +1,66 @@
package org.apache.lucene.codecs.lucene41;
/*
* 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.
*/
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.blocktree.FieldReader;
import org.apache.lucene.codecs.blocktree.Stats;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.BasePostingsFormatTestCase;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.store.Directory;
/**
* Tests BlockPostingsFormat
*/
public class TestLucene41PostingsFormat extends BasePostingsFormatTestCase {
private final Codec codec = new Lucene41RWCodec();
@Override
protected Codec getCodec() {
return codec;
}
/** Make sure the final sub-block(s) are not skipped. */
public void testFinalBlock() throws Exception {
Directory d = newDirectory();
IndexWriter w = new IndexWriter(d, new IndexWriterConfig(new MockAnalyzer(random())));
for(int i=0;i<25;i++) {
Document doc = new Document();
doc.add(newStringField("field", Character.toString((char) (97+i)), Field.Store.NO));
doc.add(newStringField("field", "z" + Character.toString((char) (97+i)), Field.Store.NO));
w.addDocument(doc);
}
w.forceMerge(1);
DirectoryReader r = DirectoryReader.open(w, true);
assertEquals(1, r.leaves().size());
FieldReader field = (FieldReader) r.leaves().get(0).reader().fields().terms("field");
// We should see exactly two blocks: one root block (prefix empty string) and one block for z* terms (prefix z):
Stats stats = field.computeStats();
assertEquals(0, stats.floorBlockCount);
assertEquals(2, stats.nonFloorBlockCount);
r.close();
w.close();
d.close();
}
}

View File

@ -0,0 +1,132 @@
package org.apache.lucene.codecs.lucene41;
/*
* 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.
*/
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
/**
* Tests special cases of BlockPostingsFormat
*/
public class TestLucene41PostingsFormat2 extends LuceneTestCase {
Directory dir;
RandomIndexWriter iw;
@Override
public void setUp() throws Exception {
super.setUp();
dir = newFSDirectory(createTempDir("testDFBlockSize"));
IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()));
iwc.setCodec(new Lucene41RWCodec());
iw = new RandomIndexWriter(random(), dir, iwc);
iw.setDoRandomForceMerge(false); // we will ourselves
}
@Override
public void tearDown() throws Exception {
iw.close();
TestUtil.checkIndex(dir); // for some extra coverage, checkIndex before we forceMerge
IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()));
iwc.setCodec(new Lucene41RWCodec());
iwc.setOpenMode(OpenMode.APPEND);
IndexWriter iw = new IndexWriter(dir, iwc);
iw.forceMerge(1);
iw.close();
dir.close(); // just force a checkindex for now
super.tearDown();
}
private Document newDocument() {
Document doc = new Document();
for (IndexOptions option : FieldInfo.IndexOptions.values()) {
FieldType ft = new FieldType(TextField.TYPE_NOT_STORED);
// turn on tvs for a cross-check, since we rely upon checkindex in this test (for now)
ft.setStoreTermVectors(true);
ft.setStoreTermVectorOffsets(true);
ft.setStoreTermVectorPositions(true);
ft.setStoreTermVectorPayloads(true);
ft.setIndexOptions(option);
doc.add(new Field(option.toString(), "", ft));
}
return doc;
}
/** tests terms with df = blocksize */
public void testDFBlockSize() throws Exception {
Document doc = newDocument();
for (int i = 0; i < Lucene41PostingsFormat.BLOCK_SIZE; i++) {
for (Field f : doc.getFields()) {
f.setStringValue(f.name() + " " + f.name() + "_2");
}
iw.addDocument(doc);
}
}
/** tests terms with df % blocksize = 0 */
public void testDFBlockSizeMultiple() throws Exception {
Document doc = newDocument();
for (int i = 0; i < Lucene41PostingsFormat.BLOCK_SIZE * 16; i++) {
for (Field f : doc.getFields()) {
f.setStringValue(f.name() + " " + f.name() + "_2");
}
iw.addDocument(doc);
}
}
/** tests terms with ttf = blocksize */
public void testTTFBlockSize() throws Exception {
Document doc = newDocument();
for (int i = 0; i < Lucene41PostingsFormat.BLOCK_SIZE/2; i++) {
for (Field f : doc.getFields()) {
f.setStringValue(f.name() + " " + f.name() + " " + f.name() + "_2 " + f.name() + "_2");
}
iw.addDocument(doc);
}
}
/** tests terms with ttf % blocksize = 0 */
public void testTTFBlockSizeMultiple() throws Exception {
Document doc = newDocument();
for (int i = 0; i < Lucene41PostingsFormat.BLOCK_SIZE/2; i++) {
for (Field f : doc.getFields()) {
String proto = (f.name() + " " + f.name() + " " + f.name() + " " + f.name() + " "
+ f.name() + "_2 " + f.name() + "_2 " + f.name() + "_2 " + f.name() + "_2");
StringBuilder val = new StringBuilder();
for (int j = 0; j < 16; j++) {
val.append(proto);
val.append(" ");
}
f.setStringValue(val.toString());
}
iw.addDocument(doc);
}
}
}

View File

@ -0,0 +1,521 @@
package org.apache.lucene.codecs.lucene41;
/*
* 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.
*/
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.Random;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.MockFixedLengthPayloadFilter;
import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.analysis.MockVariableLengthPayloadFilter;
import org.apache.lucene.analysis.TokenFilter;
import org.apache.lucene.analysis.Tokenizer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.index.TermsEnum.SeekStatus;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.English;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.automaton.AutomatonTestUtil;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.automaton.RegExp;
/**
* Tests partial enumeration (only pulling a subset of the indexed data)
*/
public class TestLucene41PostingsFormat3 extends LuceneTestCase {
static final int MAXDOC = Lucene41PostingsFormat.BLOCK_SIZE * 20;
// creates 8 fields with different options and does "duels" of fields against each other
public void test() throws Exception {
Directory dir = newDirectory();
Analyzer analyzer = new Analyzer(Analyzer.PER_FIELD_REUSE_STRATEGY) {
@Override
protected TokenStreamComponents createComponents(String fieldName) {
Tokenizer tokenizer = new MockTokenizer();
if (fieldName.contains("payloadsFixed")) {
TokenFilter filter = new MockFixedLengthPayloadFilter(new Random(0), tokenizer, 1);
return new TokenStreamComponents(tokenizer, filter);
} else if (fieldName.contains("payloadsVariable")) {
TokenFilter filter = new MockVariableLengthPayloadFilter(new Random(0), tokenizer);
return new TokenStreamComponents(tokenizer, filter);
} else {
return new TokenStreamComponents(tokenizer);
}
}
};
IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
iwc.setCodec(new Lucene41RWCodec());
// TODO we could actually add more fields implemented with different PFs
// or, just put this test into the usual rotation?
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
Document doc = new Document();
FieldType docsOnlyType = new FieldType(TextField.TYPE_NOT_STORED);
// turn this on for a cross-check
docsOnlyType.setStoreTermVectors(true);
docsOnlyType.setIndexOptions(IndexOptions.DOCS_ONLY);
FieldType docsAndFreqsType = new FieldType(TextField.TYPE_NOT_STORED);
// turn this on for a cross-check
docsAndFreqsType.setStoreTermVectors(true);
docsAndFreqsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS);
FieldType positionsType = new FieldType(TextField.TYPE_NOT_STORED);
// turn these on for a cross-check
positionsType.setStoreTermVectors(true);
positionsType.setStoreTermVectorPositions(true);
positionsType.setStoreTermVectorOffsets(true);
positionsType.setStoreTermVectorPayloads(true);
FieldType offsetsType = new FieldType(positionsType);
offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
Field field1 = new Field("field1docs", "", docsOnlyType);
Field field2 = new Field("field2freqs", "", docsAndFreqsType);
Field field3 = new Field("field3positions", "", positionsType);
Field field4 = new Field("field4offsets", "", offsetsType);
Field field5 = new Field("field5payloadsFixed", "", positionsType);
Field field6 = new Field("field6payloadsVariable", "", positionsType);
Field field7 = new Field("field7payloadsFixedOffsets", "", offsetsType);
Field field8 = new Field("field8payloadsVariableOffsets", "", offsetsType);
doc.add(field1);
doc.add(field2);
doc.add(field3);
doc.add(field4);
doc.add(field5);
doc.add(field6);
doc.add(field7);
doc.add(field8);
for (int i = 0; i < MAXDOC; i++) {
String stringValue = Integer.toString(i) + " verycommon " + English.intToEnglish(i).replace('-', ' ') + " " + TestUtil.randomSimpleString(random());
field1.setStringValue(stringValue);
field2.setStringValue(stringValue);
field3.setStringValue(stringValue);
field4.setStringValue(stringValue);
field5.setStringValue(stringValue);
field6.setStringValue(stringValue);
field7.setStringValue(stringValue);
field8.setStringValue(stringValue);
iw.addDocument(doc);
}
iw.close();
verify(dir);
TestUtil.checkIndex(dir); // for some extra coverage, checkIndex before we forceMerge
iwc = newIndexWriterConfig(analyzer);
iwc.setCodec(new Lucene41RWCodec());
iwc.setOpenMode(OpenMode.APPEND);
IndexWriter iw2 = new IndexWriter(dir, iwc);
iw2.forceMerge(1);
iw2.close();
verify(dir);
dir.close();
}
private void verify(Directory dir) throws Exception {
DirectoryReader ir = DirectoryReader.open(dir);
for (LeafReaderContext leaf : ir.leaves()) {
LeafReader leafReader = leaf.reader();
assertTerms(leafReader.terms("field1docs"), leafReader.terms("field2freqs"), true);
assertTerms(leafReader.terms("field3positions"), leafReader.terms("field4offsets"), true);
assertTerms(leafReader.terms("field4offsets"), leafReader.terms("field5payloadsFixed"), true);
assertTerms(leafReader.terms("field5payloadsFixed"), leafReader.terms("field6payloadsVariable"), true);
assertTerms(leafReader.terms("field6payloadsVariable"), leafReader.terms("field7payloadsFixedOffsets"), true);
assertTerms(leafReader.terms("field7payloadsFixedOffsets"), leafReader.terms("field8payloadsVariableOffsets"), true);
}
ir.close();
}
// following code is almost an exact dup of code from TestDuelingCodecs: sorry!
public void assertTerms(Terms leftTerms, Terms rightTerms, boolean deep) throws Exception {
if (leftTerms == null || rightTerms == null) {
assertNull(leftTerms);
assertNull(rightTerms);
return;
}
assertTermsStatistics(leftTerms, rightTerms);
// NOTE: we don't assert hasOffsets/hasPositions/hasPayloads because they are allowed to be different
TermsEnum leftTermsEnum = leftTerms.iterator(null);
TermsEnum rightTermsEnum = rightTerms.iterator(null);
assertTermsEnum(leftTermsEnum, rightTermsEnum, true);
assertTermsSeeking(leftTerms, rightTerms);
if (deep) {
int numIntersections = atLeast(3);
for (int i = 0; i < numIntersections; i++) {
String re = AutomatonTestUtil.randomRegexp(random());
CompiledAutomaton automaton = new CompiledAutomaton(new RegExp(re, RegExp.NONE).toAutomaton());
if (automaton.type == CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
// TODO: test start term too
TermsEnum leftIntersection = leftTerms.intersect(automaton, null);
TermsEnum rightIntersection = rightTerms.intersect(automaton, null);
assertTermsEnum(leftIntersection, rightIntersection, rarely());
}
}
}
}
private void assertTermsSeeking(Terms leftTerms, Terms rightTerms) throws Exception {
TermsEnum leftEnum = null;
TermsEnum rightEnum = null;
// just an upper bound
int numTests = atLeast(20);
Random random = random();
// collect this number of terms from the left side
HashSet<BytesRef> tests = new HashSet<>();
int numPasses = 0;
while (numPasses < 10 && tests.size() < numTests) {
leftEnum = leftTerms.iterator(leftEnum);
BytesRef term = null;
while ((term = leftEnum.next()) != null) {
int code = random.nextInt(10);
if (code == 0) {
// the term
tests.add(BytesRef.deepCopyOf(term));
} else if (code == 1) {
// truncated subsequence of term
term = BytesRef.deepCopyOf(term);
if (term.length > 0) {
// truncate it
term.length = random.nextInt(term.length);
}
} else if (code == 2) {
// term, but ensure a non-zero offset
byte newbytes[] = new byte[term.length+5];
System.arraycopy(term.bytes, term.offset, newbytes, 5, term.length);
tests.add(new BytesRef(newbytes, 5, term.length));
}
}
numPasses++;
}
ArrayList<BytesRef> shuffledTests = new ArrayList<>(tests);
Collections.shuffle(shuffledTests, random);
for (BytesRef b : shuffledTests) {
leftEnum = leftTerms.iterator(leftEnum);
rightEnum = rightTerms.iterator(rightEnum);
assertEquals(leftEnum.seekExact(b), rightEnum.seekExact(b));
assertEquals(leftEnum.seekExact(b), rightEnum.seekExact(b));
SeekStatus leftStatus;
SeekStatus rightStatus;
leftStatus = leftEnum.seekCeil(b);
rightStatus = rightEnum.seekCeil(b);
assertEquals(leftStatus, rightStatus);
if (leftStatus != SeekStatus.END) {
assertEquals(leftEnum.term(), rightEnum.term());
}
leftStatus = leftEnum.seekCeil(b);
rightStatus = rightEnum.seekCeil(b);
assertEquals(leftStatus, rightStatus);
if (leftStatus != SeekStatus.END) {
assertEquals(leftEnum.term(), rightEnum.term());
}
}
}
/**
* checks collection-level statistics on Terms
*/
public void assertTermsStatistics(Terms leftTerms, Terms rightTerms) throws Exception {
if (leftTerms.getDocCount() != -1 && rightTerms.getDocCount() != -1) {
assertEquals(leftTerms.getDocCount(), rightTerms.getDocCount());
}
if (leftTerms.getSumDocFreq() != -1 && rightTerms.getSumDocFreq() != -1) {
assertEquals(leftTerms.getSumDocFreq(), rightTerms.getSumDocFreq());
}
if (leftTerms.getSumTotalTermFreq() != -1 && rightTerms.getSumTotalTermFreq() != -1) {
assertEquals(leftTerms.getSumTotalTermFreq(), rightTerms.getSumTotalTermFreq());
}
if (leftTerms.size() != -1 && rightTerms.size() != -1) {
assertEquals(leftTerms.size(), rightTerms.size());
}
}
/**
* checks the terms enum sequentially
* if deep is false, it does a 'shallow' test that doesnt go down to the docsenums
*/
public void assertTermsEnum(TermsEnum leftTermsEnum, TermsEnum rightTermsEnum, boolean deep) throws Exception {
BytesRef term;
Bits randomBits = new RandomBits(MAXDOC, random().nextDouble(), random());
DocsAndPositionsEnum leftPositions = null;
DocsAndPositionsEnum rightPositions = null;
DocsEnum leftDocs = null;
DocsEnum rightDocs = null;
while ((term = leftTermsEnum.next()) != null) {
assertEquals(term, rightTermsEnum.next());
assertTermStats(leftTermsEnum, rightTermsEnum);
if (deep) {
// with payloads + off
assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.docsAndPositions(null, leftPositions),
rightPositions = rightTermsEnum.docsAndPositions(null, rightPositions));
assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.docsAndPositions(randomBits, leftPositions),
rightPositions = rightTermsEnum.docsAndPositions(randomBits, rightPositions));
assertPositionsSkipping(leftTermsEnum.docFreq(),
leftPositions = leftTermsEnum.docsAndPositions(null, leftPositions),
rightPositions = rightTermsEnum.docsAndPositions(null, rightPositions));
assertPositionsSkipping(leftTermsEnum.docFreq(),
leftPositions = leftTermsEnum.docsAndPositions(randomBits, leftPositions),
rightPositions = rightTermsEnum.docsAndPositions(randomBits, rightPositions));
// with payloads only
assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.docsAndPositions(null, leftPositions, DocsAndPositionsEnum.FLAG_PAYLOADS),
rightPositions = rightTermsEnum.docsAndPositions(null, rightPositions, DocsAndPositionsEnum.FLAG_PAYLOADS));
assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.docsAndPositions(randomBits, leftPositions, DocsAndPositionsEnum.FLAG_PAYLOADS),
rightPositions = rightTermsEnum.docsAndPositions(randomBits, rightPositions, DocsAndPositionsEnum.FLAG_PAYLOADS));
assertPositionsSkipping(leftTermsEnum.docFreq(),
leftPositions = leftTermsEnum.docsAndPositions(null, leftPositions, DocsAndPositionsEnum.FLAG_PAYLOADS),
rightPositions = rightTermsEnum.docsAndPositions(null, rightPositions, DocsAndPositionsEnum.FLAG_PAYLOADS));
assertPositionsSkipping(leftTermsEnum.docFreq(),
leftPositions = leftTermsEnum.docsAndPositions(randomBits, leftPositions, DocsAndPositionsEnum.FLAG_PAYLOADS),
rightPositions = rightTermsEnum.docsAndPositions(randomBits, rightPositions, DocsAndPositionsEnum.FLAG_PAYLOADS));
// with offsets only
assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.docsAndPositions(null, leftPositions, DocsAndPositionsEnum.FLAG_OFFSETS),
rightPositions = rightTermsEnum.docsAndPositions(null, rightPositions, DocsAndPositionsEnum.FLAG_OFFSETS));
assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.docsAndPositions(randomBits, leftPositions, DocsAndPositionsEnum.FLAG_OFFSETS),
rightPositions = rightTermsEnum.docsAndPositions(randomBits, rightPositions, DocsAndPositionsEnum.FLAG_OFFSETS));
assertPositionsSkipping(leftTermsEnum.docFreq(),
leftPositions = leftTermsEnum.docsAndPositions(null, leftPositions, DocsAndPositionsEnum.FLAG_OFFSETS),
rightPositions = rightTermsEnum.docsAndPositions(null, rightPositions, DocsAndPositionsEnum.FLAG_OFFSETS));
assertPositionsSkipping(leftTermsEnum.docFreq(),
leftPositions = leftTermsEnum.docsAndPositions(randomBits, leftPositions, DocsAndPositionsEnum.FLAG_OFFSETS),
rightPositions = rightTermsEnum.docsAndPositions(randomBits, rightPositions, DocsAndPositionsEnum.FLAG_OFFSETS));
// with positions only
assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.docsAndPositions(null, leftPositions, DocsEnum.FLAG_NONE),
rightPositions = rightTermsEnum.docsAndPositions(null, rightPositions, DocsEnum.FLAG_NONE));
assertDocsAndPositionsEnum(leftPositions = leftTermsEnum.docsAndPositions(randomBits, leftPositions, DocsEnum.FLAG_NONE),
rightPositions = rightTermsEnum.docsAndPositions(randomBits, rightPositions, DocsEnum.FLAG_NONE));
assertPositionsSkipping(leftTermsEnum.docFreq(),
leftPositions = leftTermsEnum.docsAndPositions(null, leftPositions, DocsEnum.FLAG_NONE),
rightPositions = rightTermsEnum.docsAndPositions(null, rightPositions, DocsEnum.FLAG_NONE));
assertPositionsSkipping(leftTermsEnum.docFreq(),
leftPositions = leftTermsEnum.docsAndPositions(randomBits, leftPositions, DocsEnum.FLAG_NONE),
rightPositions = rightTermsEnum.docsAndPositions(randomBits, rightPositions, DocsEnum.FLAG_NONE));
// with freqs:
assertDocsEnum(leftDocs = leftTermsEnum.docs(null, leftDocs),
rightDocs = rightTermsEnum.docs(null, rightDocs));
assertDocsEnum(leftDocs = leftTermsEnum.docs(randomBits, leftDocs),
rightDocs = rightTermsEnum.docs(randomBits, rightDocs));
// w/o freqs:
assertDocsEnum(leftDocs = leftTermsEnum.docs(null, leftDocs, DocsEnum.FLAG_NONE),
rightDocs = rightTermsEnum.docs(null, rightDocs, DocsEnum.FLAG_NONE));
assertDocsEnum(leftDocs = leftTermsEnum.docs(randomBits, leftDocs, DocsEnum.FLAG_NONE),
rightDocs = rightTermsEnum.docs(randomBits, rightDocs, DocsEnum.FLAG_NONE));
// with freqs:
assertDocsSkipping(leftTermsEnum.docFreq(),
leftDocs = leftTermsEnum.docs(null, leftDocs),
rightDocs = rightTermsEnum.docs(null, rightDocs));
assertDocsSkipping(leftTermsEnum.docFreq(),
leftDocs = leftTermsEnum.docs(randomBits, leftDocs),
rightDocs = rightTermsEnum.docs(randomBits, rightDocs));
// w/o freqs:
assertDocsSkipping(leftTermsEnum.docFreq(),
leftDocs = leftTermsEnum.docs(null, leftDocs, DocsEnum.FLAG_NONE),
rightDocs = rightTermsEnum.docs(null, rightDocs, DocsEnum.FLAG_NONE));
assertDocsSkipping(leftTermsEnum.docFreq(),
leftDocs = leftTermsEnum.docs(randomBits, leftDocs, DocsEnum.FLAG_NONE),
rightDocs = rightTermsEnum.docs(randomBits, rightDocs, DocsEnum.FLAG_NONE));
}
}
assertNull(rightTermsEnum.next());
}
/**
* checks term-level statistics
*/
public void assertTermStats(TermsEnum leftTermsEnum, TermsEnum rightTermsEnum) throws Exception {
assertEquals(leftTermsEnum.docFreq(), rightTermsEnum.docFreq());
if (leftTermsEnum.totalTermFreq() != -1 && rightTermsEnum.totalTermFreq() != -1) {
assertEquals(leftTermsEnum.totalTermFreq(), rightTermsEnum.totalTermFreq());
}
}
/**
* checks docs + freqs + positions + payloads, sequentially
*/
public void assertDocsAndPositionsEnum(DocsAndPositionsEnum leftDocs, DocsAndPositionsEnum rightDocs) throws Exception {
if (leftDocs == null || rightDocs == null) {
assertNull(leftDocs);
assertNull(rightDocs);
return;
}
assertEquals(-1, leftDocs.docID());
assertEquals(-1, rightDocs.docID());
int docid;
while ((docid = leftDocs.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
assertEquals(docid, rightDocs.nextDoc());
int freq = leftDocs.freq();
assertEquals(freq, rightDocs.freq());
for (int i = 0; i < freq; i++) {
assertEquals(leftDocs.nextPosition(), rightDocs.nextPosition());
// we don't assert offsets/payloads, they are allowed to be different
}
}
assertEquals(DocIdSetIterator.NO_MORE_DOCS, rightDocs.nextDoc());
}
/**
* checks docs + freqs, sequentially
*/
public void assertDocsEnum(DocsEnum leftDocs, DocsEnum rightDocs) throws Exception {
if (leftDocs == null) {
assertNull(rightDocs);
return;
}
assertEquals(-1, leftDocs.docID());
assertEquals(-1, rightDocs.docID());
int docid;
while ((docid = leftDocs.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
assertEquals(docid, rightDocs.nextDoc());
// we don't assert freqs, they are allowed to be different
}
assertEquals(DocIdSetIterator.NO_MORE_DOCS, rightDocs.nextDoc());
}
/**
* checks advancing docs
*/
public void assertDocsSkipping(int docFreq, DocsEnum leftDocs, DocsEnum rightDocs) throws Exception {
if (leftDocs == null) {
assertNull(rightDocs);
return;
}
int docid = -1;
int averageGap = MAXDOC / (1+docFreq);
int skipInterval = 16;
while (true) {
if (random().nextBoolean()) {
// nextDoc()
docid = leftDocs.nextDoc();
assertEquals(docid, rightDocs.nextDoc());
} else {
// advance()
int skip = docid + (int) Math.ceil(Math.abs(skipInterval + random().nextGaussian() * averageGap));
docid = leftDocs.advance(skip);
assertEquals(docid, rightDocs.advance(skip));
}
if (docid == DocIdSetIterator.NO_MORE_DOCS) {
return;
}
// we don't assert freqs, they are allowed to be different
}
}
/**
* checks advancing docs + positions
*/
public void assertPositionsSkipping(int docFreq, DocsAndPositionsEnum leftDocs, DocsAndPositionsEnum rightDocs) throws Exception {
if (leftDocs == null || rightDocs == null) {
assertNull(leftDocs);
assertNull(rightDocs);
return;
}
int docid = -1;
int averageGap = MAXDOC / (1+docFreq);
int skipInterval = 16;
while (true) {
if (random().nextBoolean()) {
// nextDoc()
docid = leftDocs.nextDoc();
assertEquals(docid, rightDocs.nextDoc());
} else {
// advance()
int skip = docid + (int) Math.ceil(Math.abs(skipInterval + random().nextGaussian() * averageGap));
docid = leftDocs.advance(skip);
assertEquals(docid, rightDocs.advance(skip));
}
if (docid == DocIdSetIterator.NO_MORE_DOCS) {
return;
}
int freq = leftDocs.freq();
assertEquals(freq, rightDocs.freq());
for (int i = 0; i < freq; i++) {
assertEquals(leftDocs.nextPosition(), rightDocs.nextPosition());
// we don't compare the payloads, its allowed that one is empty etc
}
}
}
private static class RandomBits implements Bits {
FixedBitSet bits;
RandomBits(int maxDoc, double pctLive, Random random) {
bits = new FixedBitSet(maxDoc);
for (int i = 0; i < maxDoc; i++) {
if (random.nextDouble() <= pctLive) {
bits.set(i);
}
}
}
@Override
public boolean get(int index) {
return bits.get(index);
}
@Override
public int length() {
return bits.length();
}
}
}

View File

@ -19,9 +19,11 @@ package org.apache.lucene.codecs.lucene410;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41RWPostingsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41RWStoredFieldsFormat;
import org.apache.lucene.codecs.lucene42.Lucene42RWTermVectorsFormat;
import org.apache.lucene.codecs.lucene46.Lucene46RWSegmentInfoFormat;
@ -34,6 +36,13 @@ import org.apache.lucene.codecs.lucene49.Lucene49RWNormsFormat;
@Deprecated
public final class Lucene410RWCodec extends Lucene410Codec {
private final PostingsFormat postings = new Lucene41RWPostingsFormat();
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return postings;
}
private static final DocValuesFormat docValues = new Lucene410RWDocValuesFormat();
@Override

View File

@ -26,8 +26,6 @@ import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.asserting.AssertingCodec;
import org.apache.lucene.codecs.blocktreeords.Ords41PostingsFormat;
import org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.SortedSetDocValuesField;
@ -121,17 +119,7 @@ public class TestLucene410DocValuesFormat extends BaseCompressingDocValuesFormat
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
conf.setMergeScheduler(new SerialMergeScheduler());
// set to duel against a codec which has ordinals:
final PostingsFormat pf;
switch (random().nextInt(2)) {
case 0: pf = new Lucene41WithOrds();
break;
case 1: pf = new Ords41PostingsFormat();
break;
// TODO: these don't actually support ords!
//case 2: pf = new FSTOrdPostingsFormat();
// break;
default: throw new AssertionError();
}
final PostingsFormat pf = TestUtil.getPostingsFormatWithOrds(random());
final DocValuesFormat dv = new Lucene410RWDocValuesFormat();
conf.setCodec(new AssertingCodec() {
@Override

View File

@ -20,10 +20,12 @@ package org.apache.lucene.codecs.lucene42;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40RWSegmentInfoFormat;
import org.apache.lucene.codecs.lucene41.Lucene41RWPostingsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41RWStoredFieldsFormat;
/**
@ -37,6 +39,13 @@ public final class Lucene42RWCodec extends Lucene42Codec {
private static final NormsFormat norms = new Lucene42RWNormsFormat();
private static final StoredFieldsFormat storedFields = new Lucene41RWStoredFieldsFormat();
private static final FieldInfosFormat fieldInfosFormat = new Lucene42RWFieldInfosFormat();
private final PostingsFormat postings = new Lucene41RWPostingsFormat();
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return postings;
}
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {

View File

@ -20,10 +20,12 @@ package org.apache.lucene.codecs.lucene45;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40RWSegmentInfoFormat;
import org.apache.lucene.codecs.lucene41.Lucene41RWPostingsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41RWStoredFieldsFormat;
import org.apache.lucene.codecs.lucene42.Lucene42RWFieldInfosFormat;
import org.apache.lucene.codecs.lucene42.Lucene42RWNormsFormat;
@ -36,6 +38,13 @@ import org.apache.lucene.codecs.lucene42.Lucene42RWTermVectorsFormat;
@Deprecated
public final class Lucene45RWCodec extends Lucene45Codec {
private final PostingsFormat postings = new Lucene41RWPostingsFormat();
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return postings;
}
private static final FieldInfosFormat fieldInfosFormat = new Lucene42RWFieldInfosFormat();
@Override

View File

@ -19,9 +19,11 @@ package org.apache.lucene.codecs.lucene46;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41RWPostingsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41RWStoredFieldsFormat;
import org.apache.lucene.codecs.lucene42.Lucene42RWNormsFormat;
import org.apache.lucene.codecs.lucene42.Lucene42RWTermVectorsFormat;
@ -34,6 +36,13 @@ import org.apache.lucene.codecs.lucene45.Lucene45RWDocValuesFormat;
@Deprecated
public final class Lucene46RWCodec extends Lucene46Codec {
private final PostingsFormat postings = new Lucene41RWPostingsFormat();
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return postings;
}
private static final DocValuesFormat docValues = new Lucene45RWDocValuesFormat();
@Override

View File

@ -19,9 +19,11 @@ package org.apache.lucene.codecs.lucene49;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41RWPostingsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41RWStoredFieldsFormat;
import org.apache.lucene.codecs.lucene42.Lucene42RWTermVectorsFormat;
import org.apache.lucene.codecs.lucene46.Lucene46RWSegmentInfoFormat;
@ -33,6 +35,13 @@ import org.apache.lucene.codecs.lucene46.Lucene46RWSegmentInfoFormat;
@Deprecated
public final class Lucene49RWCodec extends Lucene49Codec {
private final PostingsFormat postings = new Lucene41RWPostingsFormat();
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return postings;
}
private static final DocValuesFormat docValues = new Lucene49RWDocValuesFormat();
@Override

View File

@ -120,7 +120,7 @@ public class BlockTermsReader extends FieldsProducer {
state.segmentInfo.getId(), state.segmentSuffix);
// Have PostingsReader init itself
postingsReader.init(in);
postingsReader.init(in, state);
// NOTE: data file is too costly to verify checksum against all the bytes on open,
// but for now we at least verify proper structure of the checksum footer: which looks

View File

@ -117,7 +117,7 @@ public class BlockTermsWriter extends FieldsConsumer implements Closeable {
//System.out.println("BTW.init seg=" + state.segmentName);
postingsWriter.init(out); // have consumer write its format/header
postingsWriter.init(out, state); // have consumer write its format/header
success = true;
} finally {
if (!success) {

View File

@ -24,14 +24,14 @@ 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.lucene41.Lucene41PostingsReader;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
import org.apache.lucene.codecs.lucene50.Lucene50PostingsReader;
import org.apache.lucene.codecs.lucene50.Lucene50PostingsWriter;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.IOUtils;
/** Uses {@link OrdsBlockTreeTermsWriter} with {@link Lucene41PostingsWriter}. */
public class Ords41PostingsFormat extends PostingsFormat {
/** Uses {@link OrdsBlockTreeTermsWriter} with {@link Lucene50PostingsWriter}. */
public class BlockTreeOrdsPostingsFormat extends PostingsFormat {
private final int minTermBlockSize;
private final int maxTermBlockSize;
@ -45,7 +45,7 @@ public class Ords41PostingsFormat extends PostingsFormat {
/** Creates {@code Lucene41PostingsFormat} with default
* settings. */
public Ords41PostingsFormat() {
public BlockTreeOrdsPostingsFormat() {
this(OrdsBlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, OrdsBlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
}
@ -53,8 +53,8 @@ public class Ords41PostingsFormat extends PostingsFormat {
* values for {@code minBlockSize} and {@code
* maxBlockSize} passed to block terms dictionary.
* @see OrdsBlockTreeTermsWriter#OrdsBlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int) */
public Ords41PostingsFormat(int minTermBlockSize, int maxTermBlockSize) {
super("OrdsLucene41");
public BlockTreeOrdsPostingsFormat(int minTermBlockSize, int maxTermBlockSize) {
super("BlockTreeOrds");
this.minTermBlockSize = minTermBlockSize;
assert minTermBlockSize > 1;
this.maxTermBlockSize = maxTermBlockSize;
@ -68,7 +68,7 @@ public class Ords41PostingsFormat extends PostingsFormat {
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase postingsWriter = new Lucene41PostingsWriter(state);
PostingsWriterBase postingsWriter = new Lucene50PostingsWriter(state);
boolean success = false;
try {
@ -87,11 +87,7 @@ public class Ords41PostingsFormat extends PostingsFormat {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postingsReader = new Lucene41PostingsReader(state.directory,
state.fieldInfos,
state.segmentInfo,
state.context,
state.segmentSuffix);
PostingsReaderBase postingsReader = new Lucene50PostingsReader(state);
boolean success = false;
try {
FieldsProducer ret = new OrdsBlockTreeTermsReader(postingsReader, state);

View File

@ -94,7 +94,7 @@ public final class OrdsBlockTreeTermsReader extends FieldsProducer {
CodecUtil.checksumEntireFile(indexIn);
// Have PostingsReader init itself
postingsReader.init(in);
postingsReader.init(in, state);
// NOTE: data file is too costly to verify checksum against all the bytes on open,

View File

@ -214,7 +214,7 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
// System.out.println("BTW.init seg=" + state.segmentName);
postingsWriter.init(out); // have consumer write its format/header
postingsWriter.init(out, state); // have consumer write its format/header
success = true;
} finally {
if (!success) {

View File

@ -26,7 +26,7 @@ import java.util.TreeMap;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat; // javadocs
import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat; // javadocs
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo.IndexOptions;
@ -54,7 +54,7 @@ import org.apache.lucene.util.automaton.Transition;
// - build depth-N prefix hash?
// - or: longer dense skip lists than just next byte?
/** Wraps {@link Lucene41PostingsFormat} format for on-disk
/** Wraps {@link Lucene50PostingsFormat} format for on-disk
* storage, but then at read time loads and stores all
* terms & postings directly in RAM as byte[], int[].
*
@ -102,12 +102,12 @@ public final class DirectPostingsFormat extends PostingsFormat {
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
return PostingsFormat.forName("Lucene41").fieldsConsumer(state);
return PostingsFormat.forName("Lucene50").fieldsConsumer(state);
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
FieldsProducer postings = PostingsFormat.forName("Lucene41").fieldsProducer(state);
FieldsProducer postings = PostingsFormat.forName("Lucene50").fieldsProducer(state);
if (state.context.context != IOContext.Context.MERGE) {
FieldsProducer loadedPostings;
try {

View File

@ -25,19 +25,19 @@ 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.lucene41.Lucene41PostingsReader;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
import org.apache.lucene.codecs.lucene50.Lucene50PostingsReader;
import org.apache.lucene.codecs.lucene50.Lucene50PostingsWriter;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.IOUtils;
/**
* FSTOrd term dict + Lucene41PBF
* FSTOrd term dict + Lucene50PBF
*/
public final class FSTOrdPostingsFormat extends PostingsFormat {
public FSTOrdPostingsFormat() {
super("FSTOrd41");
super("FSTOrd50");
}
@Override
@ -47,7 +47,7 @@ public final class FSTOrdPostingsFormat extends PostingsFormat {
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase postingsWriter = new Lucene41PostingsWriter(state);
PostingsWriterBase postingsWriter = new Lucene50PostingsWriter(state);
boolean success = false;
try {
@ -63,11 +63,7 @@ public final class FSTOrdPostingsFormat extends PostingsFormat {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postingsReader = new Lucene41PostingsReader(state.directory,
state.fieldInfos,
state.segmentInfo,
state.context,
state.segmentSuffix);
PostingsReaderBase postingsReader = new Lucene50PostingsReader(state);
boolean success = false;
try {
FieldsProducer ret = new FSTOrdTermsReader(state, postingsReader);

View File

@ -103,7 +103,7 @@ public class FSTOrdTermsReader extends FieldsProducer {
CodecUtil.checksumEntireFile(blockIn);
this.postingsReader.init(blockIn);
this.postingsReader.init(blockIn, state);
seekDir(blockIn);
final FieldInfos fieldInfos = state.fieldInfos;

View File

@ -178,7 +178,7 @@ public class FSTOrdTermsWriter extends FieldsConsumer {
state.segmentInfo.getId(), state.segmentSuffix);
CodecUtil.writeSegmentHeader(blockOut, TERMS_CODEC_NAME, VERSION_CURRENT,
state.segmentInfo.getId(), state.segmentSuffix);
this.postingsWriter.init(blockOut);
this.postingsWriter.init(blockOut, state);
success = true;
} finally {
if (!success) {

View File

@ -25,19 +25,19 @@ 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.lucene41.Lucene41PostingsReader;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
import org.apache.lucene.codecs.lucene50.Lucene50PostingsReader;
import org.apache.lucene.codecs.lucene50.Lucene50PostingsWriter;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.IOUtils;
/**
* FST term dict + Lucene41PBF
* FST term dict + Lucene50PBF
*/
public final class FSTPostingsFormat extends PostingsFormat {
public FSTPostingsFormat() {
super("FST41");
super("FST50");
}
@Override
@ -47,7 +47,7 @@ public final class FSTPostingsFormat extends PostingsFormat {
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase postingsWriter = new Lucene41PostingsWriter(state);
PostingsWriterBase postingsWriter = new Lucene50PostingsWriter(state);
boolean success = false;
try {
@ -63,11 +63,7 @@ public final class FSTPostingsFormat extends PostingsFormat {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postingsReader = new Lucene41PostingsReader(state.directory,
state.fieldInfos,
state.segmentInfo,
state.context,
state.segmentSuffix);
PostingsReaderBase postingsReader = new Lucene50PostingsReader(state);
boolean success = false;
try {
FieldsProducer ret = new FSTTermsReader(state, postingsReader);

View File

@ -86,7 +86,7 @@ public class FSTTermsReader extends FieldsProducer {
FSTTermsWriter.TERMS_VERSION_CURRENT,
state.segmentInfo.getId(), state.segmentSuffix);
CodecUtil.checksumEntireFile(in);
this.postingsReader.init(in);
this.postingsReader.init(in, state);
seekDir(in);
final FieldInfos fieldInfos = state.fieldInfos;

View File

@ -145,7 +145,7 @@ public class FSTTermsWriter extends FieldsConsumer {
CodecUtil.writeSegmentHeader(out, TERMS_CODEC_NAME, TERMS_VERSION_CURRENT,
state.segmentInfo.getId(), state.segmentSuffix);
this.postingsWriter.init(out);
this.postingsWriter.init(out, state);
success = true;
} finally {
if (!success) {

View File

@ -13,7 +13,7 @@
# See the License for the specific language governing permissions and
# limitations under the License.
org.apache.lucene.codecs.blocktreeords.Ords41PostingsFormat
org.apache.lucene.codecs.blocktreeords.BlockTreeOrdsPostingsFormat
org.apache.lucene.codecs.bloom.BloomFilteringPostingsFormat
org.apache.lucene.codecs.memory.DirectPostingsFormat
org.apache.lucene.codecs.memory.FSTOrdPostingsFormat

View File

@ -18,7 +18,6 @@ package org.apache.lucene.codecs.blockterms;
*/
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds;
import org.apache.lucene.index.BasePostingsFormatTestCase;
import org.apache.lucene.util.TestUtil;
@ -26,7 +25,7 @@ import org.apache.lucene.util.TestUtil;
* Basic tests of a PF using FixedGap terms dictionary
*/
public class TestFixedGapPostingsFormat extends BasePostingsFormatTestCase {
private final Codec codec = TestUtil.alwaysPostingsFormat(new Lucene41WithOrds(TestUtil.nextInt(random(), 1, 1000)));
private final Codec codec = TestUtil.alwaysPostingsFormat(new LuceneFixedGap(TestUtil.nextInt(random(), 1, 1000)));
@Override
protected Codec getCodec() {

View File

@ -18,7 +18,7 @@ package org.apache.lucene.codecs.blockterms;
*/
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapFixedInterval;
import org.apache.lucene.codecs.blockterms.LuceneVarGapFixedInterval;
import org.apache.lucene.index.BasePostingsFormatTestCase;
import org.apache.lucene.util.TestUtil;
@ -26,7 +26,7 @@ import org.apache.lucene.util.TestUtil;
* Basic tests of a PF using VariableGap terms dictionary (fixed interval)
*/
public class TestVarGapDocFreqIntervalPostingsFormat extends BasePostingsFormatTestCase {
private final Codec codec = TestUtil.alwaysPostingsFormat(new Lucene41VarGapFixedInterval(TestUtil.nextInt(random(), 1, 1000)));
private final Codec codec = TestUtil.alwaysPostingsFormat(new LuceneVarGapFixedInterval(TestUtil.nextInt(random(), 1, 1000)));
@Override
protected Codec getCodec() {

View File

@ -18,7 +18,7 @@ package org.apache.lucene.codecs.blockterms;
*/
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapDocFreqInterval;
import org.apache.lucene.codecs.blockterms.LuceneVarGapDocFreqInterval;
import org.apache.lucene.index.BasePostingsFormatTestCase;
import org.apache.lucene.util.TestUtil;
@ -26,7 +26,7 @@ import org.apache.lucene.util.TestUtil;
* Basic tests of a PF using VariableGap terms dictionary (fixed interval, docFreq threshold)
*/
public class TestVarGapFixedIntervalPostingsFormat extends BasePostingsFormatTestCase {
private final Codec codec = TestUtil.alwaysPostingsFormat(new Lucene41VarGapDocFreqInterval(TestUtil.nextInt(random(), 1, 100), TestUtil.nextInt(random(), 1, 1000)));
private final Codec codec = TestUtil.alwaysPostingsFormat(new LuceneVarGapDocFreqInterval(TestUtil.nextInt(random(), 1, 100), TestUtil.nextInt(random(), 1, 1000)));
@Override
protected Codec getCodec() {

View File

@ -39,7 +39,7 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.TestUtil;
public class TestOrdsBlockTree extends BasePostingsFormatTestCase {
private final Codec codec = TestUtil.alwaysPostingsFormat(new Ords41PostingsFormat());
private final Codec codec = TestUtil.alwaysPostingsFormat(new BlockTreeOrdsPostingsFormat());
@Override
protected Codec getCodec() {

View File

@ -1,55 +0,0 @@
package org.apache.lucene.codecs;
/*
* 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.
*/
import java.io.IOException;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.SegmentReadState;
/**
* Provides a {@link PostingsReaderBase} and {@link
* PostingsWriterBase}.
*
* @lucene.experimental */
// TODO: find a better name; this defines the API that the
// terms dict impls use to talk to a postings impl.
// TermsDict + PostingsReader/WriterBase == PostingsConsumer/Producer
// can we clean this up and do this some other way?
// refactor some of these classes and use covariant return?
public abstract class PostingsBaseFormat {
/** Unique name that's used to retrieve this codec when
* reading the index */
public final String name;
/** Sole constructor. */
protected PostingsBaseFormat(String name) {
this.name = name;
}
/** Creates the {@link PostingsReaderBase} for this
* format. */
public abstract PostingsReaderBase postingsReaderBase(SegmentReadState state) throws IOException;
/** Creates the {@link PostingsWriterBase} for this
* format. */
public abstract PostingsWriterBase postingsWriterBase(SegmentWriteState state) throws IOException;
}

View File

@ -23,6 +23,7 @@ import java.io.IOException;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
@ -53,7 +54,7 @@ public abstract class PostingsReaderBase implements Closeable, Accountable {
/** Performs any initialization, such as reading and
* verifying the header from the provided terms
* dictionary {@link IndexInput}. */
public abstract void init(IndexInput termsIn) throws IOException;
public abstract void init(IndexInput termsIn, SegmentReadState state) throws IOException;
/** Return a newly created empty TermState */
public abstract BlockTermState newTermState() throws IOException;

View File

@ -24,6 +24,7 @@ import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter;
import org.apache.lucene.index.DocsAndPositionsEnum; // javadocs
import org.apache.lucene.index.DocsEnum; // javadocs
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexOutput;
@ -50,7 +51,7 @@ public abstract class PostingsWriterBase implements Closeable {
/** Called once after startup, before any terms have been
* added. Implementations typically write a header to
* the provided {@code termsOut}. */
public abstract void init(IndexOutput termsOut) throws IOException;
public abstract void init(IndexOutput termsOut, SegmentWriteState state) throws IOException;
/** Write all postings for one term; use the provided
* {@link TermsEnum} to pull a {@link DocsEnum} or {@link

View File

@ -24,8 +24,6 @@ import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet;
@ -72,11 +70,6 @@ public abstract class PushPostingsWriterBase extends PostingsWriterBase {
protected PushPostingsWriterBase() {
}
/** Called once after startup, before any terms have been
* added. Implementations typically write a header to
* the provided {@code termsOut}. */
public abstract void init(IndexOutput termsOut) throws IOException;
/** Return a newly created empty TermState */
public abstract BlockTermState newTermState() throws IOException;
@ -90,26 +83,11 @@ public abstract class PushPostingsWriterBase extends PostingsWriterBase {
* and will holds metadata from PBF when returned */
public abstract void finishTerm(BlockTermState state) throws IOException;
/**
* Encode metadata as long[] and byte[]. {@code absolute} controls whether
* current term is delta encoded according to latest term.
* 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(long[] longs, DataOutput out, FieldInfo fieldInfo, BlockTermState state, boolean absolute) throws IOException;
/**
* 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?
@Override
public int setField(FieldInfo fieldInfo) {
this.fieldInfo = fieldInfo;
indexOptions = fieldInfo.getIndexOptions();

View File

@ -30,12 +30,9 @@ import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
@ -96,24 +93,22 @@ public final class BlockTreeTermsReader extends FieldsProducer {
private final int version;
/** Sole constructor. */
public BlockTreeTermsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo info,
PostingsReaderBase postingsReader, IOContext ioContext,
String segmentSuffix)
public BlockTreeTermsReader(PostingsReaderBase postingsReader, SegmentReadState state)
throws IOException {
this.postingsReader = postingsReader;
this.segment = info.name;
in = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, BlockTreeTermsWriter.TERMS_EXTENSION),
ioContext);
this.segment = state.segmentInfo.name;
String termsFileName = IndexFileNames.segmentFileName(segment, state.segmentSuffix, BlockTreeTermsWriter.TERMS_EXTENSION);
in = state.directory.openInput(termsFileName, state.context);
boolean success = false;
IndexInput indexIn = null;
try {
version = readHeader(in);
indexIn = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, BlockTreeTermsWriter.TERMS_INDEX_EXTENSION),
ioContext);
String indexFileName = IndexFileNames.segmentFileName(segment, state.segmentSuffix, BlockTreeTermsWriter.TERMS_INDEX_EXTENSION);
indexIn = state.directory.openInput(indexFileName, state.context);
int indexVersion = readIndexHeader(indexIn);
if (indexVersion != version) {
throw new CorruptIndexException("mixmatched version files: " + in + "=" + version + "," + indexIn + "=" + indexVersion, indexIn);
@ -125,7 +120,7 @@ public final class BlockTreeTermsReader extends FieldsProducer {
}
// Have PostingsReader init itself
postingsReader.init(in);
postingsReader.init(in, state);
// NOTE: data file is too costly to verify checksum against all the bytes on open,
@ -158,7 +153,7 @@ public final class BlockTreeTermsReader extends FieldsProducer {
final BytesRef rootCode = new BytesRef(new byte[numBytes]);
in.readBytes(rootCode.bytes, 0, numBytes);
rootCode.length = numBytes;
final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
final FieldInfo fieldInfo = state.fieldInfos.fieldInfo(field);
if (fieldInfo == null) {
throw new CorruptIndexException("invalid field number: " + field, in);
}
@ -176,8 +171,8 @@ public final class BlockTreeTermsReader extends FieldsProducer {
} else {
minTerm = maxTerm = null;
}
if (docCount < 0 || docCount > info.getDocCount()) { // #docs with field must be <= #docs
throw new CorruptIndexException("invalid docCount: " + docCount + " maxDoc: " + info.getDocCount(), in);
if (docCount < 0 || docCount > state.segmentInfo.getDocCount()) { // #docs with field must be <= #docs
throw new CorruptIndexException("invalid docCount: " + docCount + " maxDoc: " + state.segmentInfo.getDocCount(), in);
}
if (sumDocFreq < docCount) { // #postings must be >= #docs with field
throw new CorruptIndexException("invalid sumDocFreq: " + sumDocFreq + " docCount: " + docCount, in);

View File

@ -329,7 +329,7 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
// System.out.println("BTW.init seg=" + state.segmentName);
postingsWriter.init(out); // have consumer write its format/header
postingsWriter.init(out, state); // have consumer write its format/header
success = true;
} finally {
if (!success) {

View File

@ -22,7 +22,7 @@ import java.io.PrintStream;
import java.io.UnsupportedEncodingException;
import java.util.Locale;
import org.apache.lucene.codecs.PostingsBaseFormat;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
@ -81,11 +81,11 @@ public class Stats {
public long totalBlockSuffixBytes;
/** Total number of bytes used to store term stats (not
* including what the {@link PostingsBaseFormat}
* including what the {@link PostingsReaderBase}
* stores. */
public long totalBlockStatsBytes;
/** Total bytes stored by the {@link PostingsBaseFormat},
/** Total bytes stored by the {@link PostingsReaderBase},
* plus the other few vInts stored in the frame. */
public long totalBlockOtherBytes;

View File

@ -1,51 +0,0 @@
package org.apache.lucene.codecs.lucene41;
/*
* 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.
*/
import java.io.IOException;
import org.apache.lucene.codecs.PostingsBaseFormat;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
/**
* Provides a {@link PostingsReaderBase} and {@link
* PostingsWriterBase}.
*
* @lucene.experimental */
// TODO: should these also be named / looked up via SPI?
public final class Lucene41PostingsBaseFormat extends PostingsBaseFormat {
/** Sole constructor. */
public Lucene41PostingsBaseFormat() {
super("Lucene41");
}
@Override
public PostingsReaderBase postingsReaderBase(SegmentReadState state) throws IOException {
return new Lucene41PostingsReader(state.directory, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
}
@Override
public PostingsWriterBase postingsWriterBase(SegmentWriteState state) throws IOException {
return new Lucene41PostingsWriter(state);
}
}

View File

@ -1,25 +0,0 @@
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
<!--
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.
-->
<html>
<head>
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
</head>
<body>
Lucene 4.1 file format.
</body>
</html>

View File

@ -1,4 +1,4 @@
package org.apache.lucene.codecs.lucene41;
package org.apache.lucene.codecs.lucene50;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
@ -27,7 +27,7 @@ import org.apache.lucene.util.packed.PackedInts.Decoder;
import org.apache.lucene.util.packed.PackedInts.FormatAndBits;
import org.apache.lucene.util.packed.PackedInts;
import static org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat.BLOCK_SIZE;
import static org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat.BLOCK_SIZE;
/**
* Encode all values in normal area with fixed bit width,

View File

@ -106,7 +106,7 @@ public class Lucene50Codec extends Codec {
/** Returns the postings format that should be used for writing
* new segments of <code>field</code>.
*
* The default implementation always returns "Lucene41"
* The default implementation always returns "Lucene50"
*/
public PostingsFormat getPostingsFormatForField(String field) {
return defaultFormat;
@ -126,7 +126,7 @@ public class Lucene50Codec extends Codec {
return docValuesFormat;
}
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50");
private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene50");
private final NormsFormat normsFormat = new Lucene50NormsFormat();

View File

@ -1,4 +1,4 @@
package org.apache.lucene.codecs.lucene41;
package org.apache.lucene.codecs.lucene50;
/*
@ -38,12 +38,9 @@ import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.packed.PackedInts;
/**
* Lucene 4.1 postings format, which encodes postings in packed integer blocks
* Lucene 5.0 postings format, which encodes postings in packed integer blocks
* for fast decode.
*
* <p><b>NOTE</b>: this format is still experimental and
* subject to change without backwards compatibility.
*
* <p>
* Basic idea:
* <ul>
@ -58,7 +55,7 @@ import org.apache.lucene.util.packed.PackedInts;
*
* <li>
* <b>Block structure</b>:
* <p>When the postings are long enough, Lucene41PostingsFormat will try to encode most integer data
* <p>When the postings are long enough, Lucene50PostingsFormat will try to encode most integer data
* as a packed block.</p>
* <p>Take a term with 259 documents as an example, the first 256 document ids are encoded as two packed
* blocks, while the remaining 3 are encoded as one VInt block. </p>
@ -129,14 +126,14 @@ import org.apache.lucene.util.packed.PackedInts;
* <li>PostingsHeader --&gt; Header, PackedBlockSize</li>
* <li>TermMetadata --&gt; (DocFPDelta|SingletonDocID), PosFPDelta?, PosVIntBlockFPDelta?, PayFPDelta?,
* SkipFPDelta?</li>
* <li>Header, --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
* <li>Header, --&gt; {@link CodecUtil#writeSegmentHeader SegmentHeader}</li>
* <li>PackedBlockSize, SingletonDocID --&gt; {@link DataOutput#writeVInt VInt}</li>
* <li>DocFPDelta, PosFPDelta, PayFPDelta, PosVIntBlockFPDelta, SkipFPDelta --&gt; {@link DataOutput#writeVLong VLong}</li>
* <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
* </ul>
* <p>Notes:</p>
* <ul>
* <li>Header is a {@link CodecUtil#writeHeader CodecHeader} storing the version information
* <li>Header is a {@link CodecUtil#writeSegmentHeader SegmentHeader} storing the version information
* for the postings.</li>
* <li>PackedBlockSize is the fixed block size for packed blocks. In packed block, bit width is
* determined by the largest integer. Smaller block size result in smaller variance among width
@ -162,7 +159,7 @@ import org.apache.lucene.util.packed.PackedInts;
* <li>SkipFPDelta determines the position of this term's SkipData within the .doc
* file. In particular, it is the length of the TermFreq data.
* SkipDelta is only stored if DocFreq is not smaller than SkipMinimum
* (i.e. 128 in Lucene41PostingsFormat).</li>
* (i.e. 128 in Lucene50PostingsFormat).</li>
* <li>SingletonDocID is an optimization when a term only appears in one document. In this case, instead
* of writing a file pointer to the .doc file (DocFPDelta), and then a VIntBlock at that location, the
* single document ID is written to the term dictionary.</li>
@ -192,7 +189,7 @@ import org.apache.lucene.util.packed.PackedInts;
*
* <ul>
* <li>docFile(.doc) --&gt; Header, &lt;TermFreqs, SkipData?&gt;<sup>TermCount</sup>, Footer</li>
* <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
* <li>Header --&gt; {@link CodecUtil#writeSegmentHeader SegmentHeader}</li>
* <li>TermFreqs --&gt; &lt;PackedBlock&gt; <sup>PackedDocBlockNum</sup>,
* VIntBlock? </li>
* <li>PackedBlock --&gt; PackedDocDeltaBlock, PackedFreqBlock?
@ -243,10 +240,10 @@ import org.apache.lucene.util.packed.PackedInts;
* We use this trick since the definition of skip entry is a little different from base interface.
* In {@link MultiLevelSkipListWriter}, skip data is assumed to be saved for
* skipInterval<sup>th</sup>, 2*skipInterval<sup>th</sup> ... posting in the list. However,
* in Lucene41PostingsFormat, the skip data is saved for skipInterval+1<sup>th</sup>,
* in Lucene50PostingsFormat, the skip data is saved for skipInterval+1<sup>th</sup>,
* 2*skipInterval+1<sup>th</sup> ... posting (skipInterval==PackedBlockSize in this case).
* When DocFreq is multiple of PackedBlockSize, MultiLevelSkipListWriter will expect one
* more skip data than Lucene41SkipWriter. </li>
* more skip data than Lucene50SkipWriter. </li>
* <li>SkipDatum is the metadata of one skip entry.
* For the first block (no matter packed or VInt), it is omitted.</li>
* <li>DocSkip records the document number of every PackedBlockSize<sup>th</sup> document number in
@ -276,7 +273,7 @@ import org.apache.lucene.util.packed.PackedInts;
* sometimes stores part of payloads and offsets for speedup.</p>
* <ul>
* <li>PosFile(.pos) --&gt; Header, &lt;TermPositions&gt; <sup>TermCount</sup>, Footer</li>
* <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
* <li>Header --&gt; {@link CodecUtil#writeSegmentHeader SegmentHeader}</li>
* <li>TermPositions --&gt; &lt;PackedPosDeltaBlock&gt; <sup>PackedPosBlockNum</sup>,
* VIntBlock? </li>
* <li>VIntBlock --&gt; &lt;PositionDelta[, PayloadLength?], PayloadData?,
@ -329,7 +326,7 @@ import org.apache.lucene.util.packed.PackedInts;
* Some payloads and offsets will be separated out into .pos file, for performance reasons.</p>
* <ul>
* <li>PayFile(.pay): --&gt; Header, &lt;TermPayloads, TermOffsets?&gt; <sup>TermCount</sup>, Footer</li>
* <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
* <li>Header --&gt; {@link CodecUtil#writeSegmentHeader SegmentHeader}</li>
* <li>TermPayloads --&gt; &lt;PackedPayLengthBlock, SumPayLength, PayData&gt; <sup>PackedPayBlockNum</sup>
* <li>TermOffsets --&gt; &lt;PackedOffsetStartDeltaBlock, PackedOffsetLengthBlock&gt; <sup>PackedPayBlockNum</sup>
* <li>PackedPayLengthBlock, PackedOffsetStartDeltaBlock, PackedOffsetLengthBlock --&gt; {@link PackedInts PackedInts}</li>
@ -358,7 +355,7 @@ import org.apache.lucene.util.packed.PackedInts;
* @lucene.experimental
*/
public final class Lucene41PostingsFormat extends PostingsFormat {
public final class Lucene50PostingsFormat extends PostingsFormat {
/**
* Filename extension for document number, frequencies, and skip data.
* See chapter: <a href="#Frequencies">Frequencies and Skip Data</a>
@ -387,18 +384,18 @@ public final class Lucene41PostingsFormat extends PostingsFormat {
// NOTE: must be multiple of 64 because of PackedInts long-aligned encoding/decoding
public final static int BLOCK_SIZE = 128;
/** Creates {@code Lucene41PostingsFormat} with default
/** Creates {@code Lucene50PostingsFormat} with default
* settings. */
public Lucene41PostingsFormat() {
public Lucene50PostingsFormat() {
this(BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
}
/** Creates {@code Lucene41PostingsFormat} with custom
/** Creates {@code Lucene50PostingsFormat} with custom
* values for {@code minBlockSize} and {@code
* maxBlockSize} passed to block terms dictionary.
* @see BlockTreeTermsWriter#BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int) */
public Lucene41PostingsFormat(int minTermBlockSize, int maxTermBlockSize) {
super("Lucene41");
public Lucene50PostingsFormat(int minTermBlockSize, int maxTermBlockSize) {
super("Lucene50");
this.minTermBlockSize = minTermBlockSize;
assert minTermBlockSize > 1;
this.maxTermBlockSize = maxTermBlockSize;
@ -412,7 +409,7 @@ public final class Lucene41PostingsFormat extends PostingsFormat {
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase postingsWriter = new Lucene41PostingsWriter(state);
PostingsWriterBase postingsWriter = new Lucene50PostingsWriter(state);
boolean success = false;
try {
@ -431,19 +428,10 @@ public final class Lucene41PostingsFormat extends PostingsFormat {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postingsReader = new Lucene41PostingsReader(state.directory,
state.fieldInfos,
state.segmentInfo,
state.context,
state.segmentSuffix);
PostingsReaderBase postingsReader = new Lucene50PostingsReader(state);
boolean success = false;
try {
FieldsProducer ret = new BlockTreeTermsReader(state.directory,
state.fieldInfos,
state.segmentInfo,
postingsReader,
state.context,
state.segmentSuffix);
FieldsProducer ret = new BlockTreeTermsReader(postingsReader, state);
success = true;
return ret;
} finally {

View File

@ -0,0 +1,525 @@
package org.apache.lucene.codecs.lucene50;
/*
* 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.
*/
import java.io.IOException;
import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.PushPostingsWriterBase;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.TermState;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.packed.PackedInts;
import static org.apache.lucene.codecs.lucene50.ForUtil.MAX_DATA_SIZE;
import static org.apache.lucene.codecs.lucene50.ForUtil.MAX_ENCODED_SIZE;
import static org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat.BLOCK_SIZE;
/**
* Concrete class that writes docId(maybe frq,pos,offset,payloads) list
* with postings format.
*
* Postings list for each term will be stored separately.
*
* @see Lucene50SkipWriter for details about skipping setting and postings layout.
* @lucene.experimental
*/
public final class Lucene50PostingsWriter extends PushPostingsWriterBase {
/**
* Expert: The maximum number of skip levels. Smaller values result in
* slightly smaller indexes, but slower skipping in big posting lists.
*/
static final int MAX_SKIP_LEVELS = 10;
final static String TERMS_CODEC = "Lucene50PostingsWriterTerms";
final static String DOC_CODEC = "Lucene50PostingsWriterDoc";
final static String POS_CODEC = "Lucene50PostingsWriterPos";
final static String PAY_CODEC = "Lucene50PostingsWriterPay";
// Increment version to change it
final static int VERSION_START = 0;
final static int VERSION_CURRENT = VERSION_START;
IndexOutput docOut;
IndexOutput posOut;
IndexOutput payOut;
final static IntBlockTermState emptyState = new IntBlockTermState();
IntBlockTermState lastState;
// Holds starting file pointers for current term:
private long docStartFP;
private long posStartFP;
private long payStartFP;
final int[] docDeltaBuffer;
final int[] freqBuffer;
private int docBufferUpto;
final int[] posDeltaBuffer;
final int[] payloadLengthBuffer;
final int[] offsetStartDeltaBuffer;
final int[] offsetLengthBuffer;
private int posBufferUpto;
private byte[] payloadBytes;
private int payloadByteUpto;
private int lastBlockDocID;
private long lastBlockPosFP;
private long lastBlockPayFP;
private int lastBlockPosBufferUpto;
private int lastBlockPayloadByteUpto;
private int lastDocID;
private int lastPosition;
private int lastStartOffset;
private int docCount;
final byte[] encoded;
private final ForUtil forUtil;
private final Lucene50SkipWriter skipWriter;
/** Creates a postings writer */
public Lucene50PostingsWriter(SegmentWriteState state) throws IOException {
final float acceptableOverheadRatio = PackedInts.COMPACT;
String docFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene50PostingsFormat.DOC_EXTENSION);
docOut = state.directory.createOutput(docFileName, state.context);
IndexOutput posOut = null;
IndexOutput payOut = null;
boolean success = false;
try {
CodecUtil.writeSegmentHeader(docOut, DOC_CODEC, VERSION_CURRENT,
state.segmentInfo.getId(), state.segmentSuffix);
forUtil = new ForUtil(acceptableOverheadRatio, docOut);
if (state.fieldInfos.hasProx()) {
posDeltaBuffer = new int[MAX_DATA_SIZE];
String posFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene50PostingsFormat.POS_EXTENSION);
posOut = state.directory.createOutput(posFileName, state.context);
CodecUtil.writeSegmentHeader(posOut, POS_CODEC, VERSION_CURRENT,
state.segmentInfo.getId(), state.segmentSuffix);
if (state.fieldInfos.hasPayloads()) {
payloadBytes = new byte[128];
payloadLengthBuffer = new int[MAX_DATA_SIZE];
} else {
payloadBytes = null;
payloadLengthBuffer = null;
}
if (state.fieldInfos.hasOffsets()) {
offsetStartDeltaBuffer = new int[MAX_DATA_SIZE];
offsetLengthBuffer = new int[MAX_DATA_SIZE];
} else {
offsetStartDeltaBuffer = null;
offsetLengthBuffer = null;
}
if (state.fieldInfos.hasPayloads() || state.fieldInfos.hasOffsets()) {
String payFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene50PostingsFormat.PAY_EXTENSION);
payOut = state.directory.createOutput(payFileName, state.context);
CodecUtil.writeSegmentHeader(payOut, PAY_CODEC, VERSION_CURRENT,
state.segmentInfo.getId(), state.segmentSuffix);
}
} else {
posDeltaBuffer = null;
payloadLengthBuffer = null;
offsetStartDeltaBuffer = null;
offsetLengthBuffer = null;
payloadBytes = null;
}
this.payOut = payOut;
this.posOut = posOut;
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(docOut, posOut, payOut);
}
}
docDeltaBuffer = new int[MAX_DATA_SIZE];
freqBuffer = new int[MAX_DATA_SIZE];
// TODO: should we try skipping every 2/4 blocks...?
skipWriter = new Lucene50SkipWriter(MAX_SKIP_LEVELS,
BLOCK_SIZE,
state.segmentInfo.getDocCount(),
docOut,
posOut,
payOut);
encoded = new byte[MAX_ENCODED_SIZE];
}
final static class IntBlockTermState extends BlockTermState {
long docStartFP = 0;
long posStartFP = 0;
long payStartFP = 0;
long skipOffset = -1;
long lastPosBlockOffset = -1;
// docid when there is a single pulsed posting, otherwise -1
// freq is always implicitly totalTermFreq in this case.
int singletonDocID = -1;
@Override
public IntBlockTermState clone() {
IntBlockTermState other = new IntBlockTermState();
other.copyFrom(this);
return other;
}
@Override
public void copyFrom(TermState _other) {
super.copyFrom(_other);
IntBlockTermState other = (IntBlockTermState) _other;
docStartFP = other.docStartFP;
posStartFP = other.posStartFP;
payStartFP = other.payStartFP;
lastPosBlockOffset = other.lastPosBlockOffset;
skipOffset = other.skipOffset;
singletonDocID = other.singletonDocID;
}
@Override
public String toString() {
return super.toString() + " docStartFP=" + docStartFP + " posStartFP=" + posStartFP + " payStartFP=" + payStartFP + " lastPosBlockOffset=" + lastPosBlockOffset + " singletonDocID=" + singletonDocID;
}
}
@Override
public IntBlockTermState newTermState() {
return new IntBlockTermState();
}
@Override
public void init(IndexOutput termsOut, SegmentWriteState state) throws IOException {
CodecUtil.writeSegmentHeader(termsOut, TERMS_CODEC, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
termsOut.writeVInt(BLOCK_SIZE);
}
@Override
public int setField(FieldInfo fieldInfo) {
super.setField(fieldInfo);
skipWriter.setField(writePositions, writeOffsets, writePayloads);
lastState = emptyState;
if (writePositions) {
if (writePayloads || writeOffsets) {
return 3; // doc + pos + pay FP
} else {
return 2; // doc + pos FP
}
} else {
return 1; // doc FP
}
}
@Override
public void startTerm() {
docStartFP = docOut.getFilePointer();
if (writePositions) {
posStartFP = posOut.getFilePointer();
if (writePayloads || writeOffsets) {
payStartFP = payOut.getFilePointer();
}
}
lastDocID = 0;
lastBlockDocID = -1;
skipWriter.resetSkip();
}
@Override
public void startDoc(int docID, int termDocFreq) throws IOException {
// Have collected a block of docs, and get a new doc.
// Should write skip data as well as postings list for
// current block.
if (lastBlockDocID != -1 && docBufferUpto == 0) {
skipWriter.bufferSkip(lastBlockDocID, docCount, lastBlockPosFP, lastBlockPayFP, lastBlockPosBufferUpto, lastBlockPayloadByteUpto);
}
final int docDelta = docID - lastDocID;
if (docID < 0 || (docCount > 0 && docDelta <= 0)) {
throw new CorruptIndexException("docs out of order (" + docID + " <= " + lastDocID + " )", docOut.toString());
}
docDeltaBuffer[docBufferUpto] = docDelta;
if (writeFreqs) {
freqBuffer[docBufferUpto] = termDocFreq;
}
docBufferUpto++;
docCount++;
if (docBufferUpto == BLOCK_SIZE) {
forUtil.writeBlock(docDeltaBuffer, encoded, docOut);
if (writeFreqs) {
forUtil.writeBlock(freqBuffer, encoded, docOut);
}
// NOTE: don't set docBufferUpto back to 0 here;
// finishDoc will do so (because it needs to see that
// the block was filled so it can save skip data)
}
lastDocID = docID;
lastPosition = 0;
lastStartOffset = 0;
}
@Override
public void addPosition(int position, BytesRef payload, int startOffset, int endOffset) throws IOException {
posDeltaBuffer[posBufferUpto] = position - lastPosition;
if (writePayloads) {
if (payload == null || payload.length == 0) {
// no payload
payloadLengthBuffer[posBufferUpto] = 0;
} else {
payloadLengthBuffer[posBufferUpto] = payload.length;
if (payloadByteUpto + payload.length > payloadBytes.length) {
payloadBytes = ArrayUtil.grow(payloadBytes, payloadByteUpto + payload.length);
}
System.arraycopy(payload.bytes, payload.offset, payloadBytes, payloadByteUpto, payload.length);
payloadByteUpto += payload.length;
}
}
if (writeOffsets) {
assert startOffset >= lastStartOffset;
assert endOffset >= startOffset;
offsetStartDeltaBuffer[posBufferUpto] = startOffset - lastStartOffset;
offsetLengthBuffer[posBufferUpto] = endOffset - startOffset;
lastStartOffset = startOffset;
}
posBufferUpto++;
lastPosition = position;
if (posBufferUpto == BLOCK_SIZE) {
forUtil.writeBlock(posDeltaBuffer, encoded, posOut);
if (writePayloads) {
forUtil.writeBlock(payloadLengthBuffer, encoded, payOut);
payOut.writeVInt(payloadByteUpto);
payOut.writeBytes(payloadBytes, 0, payloadByteUpto);
payloadByteUpto = 0;
}
if (writeOffsets) {
forUtil.writeBlock(offsetStartDeltaBuffer, encoded, payOut);
forUtil.writeBlock(offsetLengthBuffer, encoded, payOut);
}
posBufferUpto = 0;
}
}
@Override
public void finishDoc() throws IOException {
// Since we don't know df for current term, we had to buffer
// those skip data for each block, and when a new doc comes,
// write them to skip file.
if (docBufferUpto == BLOCK_SIZE) {
lastBlockDocID = lastDocID;
if (posOut != null) {
if (payOut != null) {
lastBlockPayFP = payOut.getFilePointer();
}
lastBlockPosFP = posOut.getFilePointer();
lastBlockPosBufferUpto = posBufferUpto;
lastBlockPayloadByteUpto = payloadByteUpto;
}
docBufferUpto = 0;
}
}
/** Called when we are done adding docs to this term */
@Override
public void finishTerm(BlockTermState _state) throws IOException {
IntBlockTermState state = (IntBlockTermState) _state;
assert state.docFreq > 0;
// TODO: wasteful we are counting this (counting # docs
// for this term) in two places?
assert state.docFreq == docCount: state.docFreq + " vs " + docCount;
// docFreq == 1, don't write the single docid/freq to a separate file along with a pointer to it.
final int singletonDocID;
if (state.docFreq == 1) {
// pulse the singleton docid into the term dictionary, freq is implicitly totalTermFreq
singletonDocID = docDeltaBuffer[0];
} else {
singletonDocID = -1;
// vInt encode the remaining doc deltas and freqs:
for(int i=0;i<docBufferUpto;i++) {
final int docDelta = docDeltaBuffer[i];
final int freq = freqBuffer[i];
if (!writeFreqs) {
docOut.writeVInt(docDelta);
} else if (freqBuffer[i] == 1) {
docOut.writeVInt((docDelta<<1)|1);
} else {
docOut.writeVInt(docDelta<<1);
docOut.writeVInt(freq);
}
}
}
final long lastPosBlockOffset;
if (writePositions) {
// totalTermFreq is just total number of positions(or payloads, or offsets)
// associated with current term.
assert state.totalTermFreq != -1;
if (state.totalTermFreq > BLOCK_SIZE) {
// record file offset for last pos in last block
lastPosBlockOffset = posOut.getFilePointer() - posStartFP;
} else {
lastPosBlockOffset = -1;
}
if (posBufferUpto > 0) {
// TODO: should we send offsets/payloads to
// .pay...? seems wasteful (have to store extra
// vLong for low (< BLOCK_SIZE) DF terms = vast vast
// majority)
// vInt encode the remaining positions/payloads/offsets:
int lastPayloadLength = -1; // force first payload length to be written
int lastOffsetLength = -1; // force first offset length to be written
int payloadBytesReadUpto = 0;
for(int i=0;i<posBufferUpto;i++) {
final int posDelta = posDeltaBuffer[i];
if (writePayloads) {
final int payloadLength = payloadLengthBuffer[i];
if (payloadLength != lastPayloadLength) {
lastPayloadLength = payloadLength;
posOut.writeVInt((posDelta<<1)|1);
posOut.writeVInt(payloadLength);
} else {
posOut.writeVInt(posDelta<<1);
}
if (payloadLength != 0) {
posOut.writeBytes(payloadBytes, payloadBytesReadUpto, payloadLength);
payloadBytesReadUpto += payloadLength;
}
} else {
posOut.writeVInt(posDelta);
}
if (writeOffsets) {
int delta = offsetStartDeltaBuffer[i];
int length = offsetLengthBuffer[i];
if (length == lastOffsetLength) {
posOut.writeVInt(delta << 1);
} else {
posOut.writeVInt(delta << 1 | 1);
posOut.writeVInt(length);
lastOffsetLength = length;
}
}
}
if (writePayloads) {
assert payloadBytesReadUpto == payloadByteUpto;
payloadByteUpto = 0;
}
}
} else {
lastPosBlockOffset = -1;
}
long skipOffset;
if (docCount > BLOCK_SIZE) {
skipOffset = skipWriter.writeSkip(docOut) - docStartFP;
} else {
skipOffset = -1;
}
state.docStartFP = docStartFP;
state.posStartFP = posStartFP;
state.payStartFP = payStartFP;
state.singletonDocID = singletonDocID;
state.skipOffset = skipOffset;
state.lastPosBlockOffset = lastPosBlockOffset;
docBufferUpto = 0;
posBufferUpto = 0;
lastDocID = 0;
docCount = 0;
}
@Override
public void encodeTerm(long[] longs, DataOutput out, FieldInfo fieldInfo, BlockTermState _state, boolean absolute) throws IOException {
IntBlockTermState state = (IntBlockTermState)_state;
if (absolute) {
lastState = emptyState;
}
longs[0] = state.docStartFP - lastState.docStartFP;
if (writePositions) {
longs[1] = state.posStartFP - lastState.posStartFP;
if (writePayloads || writeOffsets) {
longs[2] = state.payStartFP - lastState.payStartFP;
}
}
if (state.singletonDocID != -1) {
out.writeVInt(state.singletonDocID);
}
if (writePositions) {
if (state.lastPosBlockOffset != -1) {
out.writeVLong(state.lastPosBlockOffset);
}
}
if (state.skipOffset != -1) {
out.writeVLong(state.skipOffset);
}
lastState = state;
}
@Override
public void close() throws IOException {
// TODO: add a finish() at least to PushBase? DV too...?
boolean success = false;
try {
if (docOut != null) {
CodecUtil.writeFooter(docOut);
}
if (posOut != null) {
CodecUtil.writeFooter(posOut);
}
if (payOut != null) {
CodecUtil.writeFooter(payOut);
}
success = true;
} finally {
if (success) {
IOUtils.close(docOut, posOut, payOut);
} else {
IOUtils.closeWhileHandlingException(docOut, posOut, payOut);
}
docOut = posOut = payOut = null;
}
}
}

View File

@ -0,0 +1,198 @@
package org.apache.lucene.codecs.lucene50;
/*
* 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.
*/
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.codecs.MultiLevelSkipListReader;
import org.apache.lucene.store.IndexInput;
/**
* Implements the skip list reader for block postings format
* that stores positions and payloads.
*
* Although this skipper uses MultiLevelSkipListReader as an interface,
* its definition of skip position will be a little different.
*
* For example, when skipInterval = blockSize = 3, df = 2*skipInterval = 6,
*
* 0 1 2 3 4 5
* d d d d d d (posting list)
* ^ ^ (skip point in MultiLeveSkipWriter)
* ^ (skip point in Lucene50SkipWriter)
*
* In this case, MultiLevelSkipListReader will use the last document as a skip point,
* while Lucene50SkipReader should assume no skip point will comes.
*
* If we use the interface directly in Lucene50SkipReader, it may silly try to read
* another skip data after the only skip point is loaded.
*
* To illustrate this, we can call skipTo(d[5]), since skip point d[3] has smaller docId,
* and numSkipped+blockSize== df, the MultiLevelSkipListReader will assume the skip list
* isn't exhausted yet, and try to load a non-existed skip point
*
* Therefore, we'll trim df before passing it to the interface. see trim(int)
*
*/
final class Lucene50SkipReader extends MultiLevelSkipListReader {
private final int blockSize;
private long docPointer[];
private long posPointer[];
private long payPointer[];
private int posBufferUpto[];
private int payloadByteUpto[];
private long lastPosPointer;
private long lastPayPointer;
private int lastPayloadByteUpto;
private long lastDocPointer;
private int lastPosBufferUpto;
public Lucene50SkipReader(IndexInput skipStream, int maxSkipLevels, int blockSize, boolean hasPos, boolean hasOffsets, boolean hasPayloads) {
super(skipStream, maxSkipLevels, blockSize, 8);
this.blockSize = blockSize;
docPointer = new long[maxSkipLevels];
if (hasPos) {
posPointer = new long[maxSkipLevels];
posBufferUpto = new int[maxSkipLevels];
if (hasPayloads) {
payloadByteUpto = new int[maxSkipLevels];
} else {
payloadByteUpto = null;
}
if (hasOffsets || hasPayloads) {
payPointer = new long[maxSkipLevels];
} else {
payPointer = null;
}
} else {
posPointer = null;
}
}
/**
* Trim original docFreq to tell skipReader read proper number of skip points.
*
* Since our definition in Lucene50Skip* is a little different from MultiLevelSkip*
* This trimmed docFreq will prevent skipReader from:
* 1. silly reading a non-existed skip point after the last block boundary
* 2. moving into the vInt block
*
*/
protected int trim(int df) {
return df % blockSize == 0? df - 1: df;
}
public void init(long skipPointer, long docBasePointer, long posBasePointer, long payBasePointer, int df) {
super.init(skipPointer, trim(df));
lastDocPointer = docBasePointer;
lastPosPointer = posBasePointer;
lastPayPointer = payBasePointer;
Arrays.fill(docPointer, docBasePointer);
if (posPointer != null) {
Arrays.fill(posPointer, posBasePointer);
if (payPointer != null) {
Arrays.fill(payPointer, payBasePointer);
}
} else {
assert posBasePointer == 0;
}
}
/** Returns the doc pointer of the doc to which the last call of
* {@link MultiLevelSkipListReader#skipTo(int)} has skipped. */
public long getDocPointer() {
return lastDocPointer;
}
public long getPosPointer() {
return lastPosPointer;
}
public int getPosBufferUpto() {
return lastPosBufferUpto;
}
public long getPayPointer() {
return lastPayPointer;
}
public int getPayloadByteUpto() {
return lastPayloadByteUpto;
}
public int getNextSkipDoc() {
return skipDoc[0];
}
@Override
protected void seekChild(int level) throws IOException {
super.seekChild(level);
docPointer[level] = lastDocPointer;
if (posPointer != null) {
posPointer[level] = lastPosPointer;
posBufferUpto[level] = lastPosBufferUpto;
if (payloadByteUpto != null) {
payloadByteUpto[level] = lastPayloadByteUpto;
}
if (payPointer != null) {
payPointer[level] = lastPayPointer;
}
}
}
@Override
protected void setLastSkipData(int level) {
super.setLastSkipData(level);
lastDocPointer = docPointer[level];
if (posPointer != null) {
lastPosPointer = posPointer[level];
lastPosBufferUpto = posBufferUpto[level];
if (payPointer != null) {
lastPayPointer = payPointer[level];
}
if (payloadByteUpto != null) {
lastPayloadByteUpto = payloadByteUpto[level];
}
}
}
@Override
protected int readSkipData(int level, IndexInput skipStream) throws IOException {
int delta = skipStream.readVInt();
docPointer[level] += skipStream.readVInt();
if (posPointer != null) {
posPointer[level] += skipStream.readVInt();
posBufferUpto[level] = skipStream.readVInt();
if (payloadByteUpto != null) {
payloadByteUpto[level] = skipStream.readVInt();
}
if (payPointer != null) {
payPointer[level] += skipStream.readVInt();
}
}
return delta;
}
}

View File

@ -0,0 +1,169 @@
package org.apache.lucene.codecs.lucene50;
/*
* 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.
*/
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.codecs.MultiLevelSkipListWriter;
/**
* Write skip lists with multiple levels, and support skip within block ints.
*
* Assume that docFreq = 28, skipInterval = blockSize = 12
*
* | block#0 | | block#1 | |vInts|
* d d d d d d d d d d d d d d d d d d d d d d d d d d d d (posting list)
* ^ ^ (level 0 skip point)
*
* Note that skipWriter will ignore first document in block#0, since
* it is useless as a skip point. Also, we'll never skip into the vInts
* block, only record skip data at the start its start point(if it exist).
*
* For each skip point, we will record:
* 1. docID in former position, i.e. for position 12, record docID[11], etc.
* 2. its related file points(position, payload),
* 3. related numbers or uptos(position, payload).
* 4. start offset.
*
*/
final class Lucene50SkipWriter extends MultiLevelSkipListWriter {
private int[] lastSkipDoc;
private long[] lastSkipDocPointer;
private long[] lastSkipPosPointer;
private long[] lastSkipPayPointer;
private int[] lastPayloadByteUpto;
private final IndexOutput docOut;
private final IndexOutput posOut;
private final IndexOutput payOut;
private int curDoc;
private long curDocPointer;
private long curPosPointer;
private long curPayPointer;
private int curPosBufferUpto;
private int curPayloadByteUpto;
private boolean fieldHasPositions;
private boolean fieldHasOffsets;
private boolean fieldHasPayloads;
public Lucene50SkipWriter(int maxSkipLevels, int blockSize, int docCount, IndexOutput docOut, IndexOutput posOut, IndexOutput payOut) {
super(blockSize, 8, maxSkipLevels, docCount);
this.docOut = docOut;
this.posOut = posOut;
this.payOut = payOut;
lastSkipDoc = new int[maxSkipLevels];
lastSkipDocPointer = new long[maxSkipLevels];
if (posOut != null) {
lastSkipPosPointer = new long[maxSkipLevels];
if (payOut != null) {
lastSkipPayPointer = new long[maxSkipLevels];
}
lastPayloadByteUpto = new int[maxSkipLevels];
}
}
public void setField(boolean fieldHasPositions, boolean fieldHasOffsets, boolean fieldHasPayloads) {
this.fieldHasPositions = fieldHasPositions;
this.fieldHasOffsets = fieldHasOffsets;
this.fieldHasPayloads = fieldHasPayloads;
}
// tricky: we only skip data for blocks (terms with more than 128 docs), but re-init'ing the skipper
// is pretty slow for rare terms in large segments as we have to fill O(log #docs in segment) of junk.
// this is the vast majority of terms (worst case: ID field or similar). so in resetSkip() we save
// away the previous pointers, and lazy-init only if we need to buffer skip data for the term.
private boolean initialized;
long lastDocFP;
long lastPosFP;
long lastPayFP;
@Override
public void resetSkip() {
lastDocFP = docOut.getFilePointer();
if (fieldHasPositions) {
lastPosFP = posOut.getFilePointer();
if (fieldHasOffsets || fieldHasPayloads) {
lastPayFP = payOut.getFilePointer();
}
}
initialized = false;
}
public void initSkip() {
if (!initialized) {
super.resetSkip();
Arrays.fill(lastSkipDoc, 0);
Arrays.fill(lastSkipDocPointer, lastDocFP);
if (fieldHasPositions) {
Arrays.fill(lastSkipPosPointer, lastPosFP);
if (fieldHasPayloads) {
Arrays.fill(lastPayloadByteUpto, 0);
}
if (fieldHasOffsets || fieldHasPayloads) {
Arrays.fill(lastSkipPayPointer, lastPayFP);
}
}
initialized = true;
}
}
/**
* Sets the values for the current skip data.
*/
public void bufferSkip(int doc, int numDocs, long posFP, long payFP, int posBufferUpto, int payloadByteUpto) throws IOException {
initSkip();
this.curDoc = doc;
this.curDocPointer = docOut.getFilePointer();
this.curPosPointer = posFP;
this.curPayPointer = payFP;
this.curPosBufferUpto = posBufferUpto;
this.curPayloadByteUpto = payloadByteUpto;
bufferSkip(numDocs);
}
@Override
protected void writeSkipData(int level, IndexOutput skipBuffer) throws IOException {
int delta = curDoc - lastSkipDoc[level];
skipBuffer.writeVInt(delta);
lastSkipDoc[level] = curDoc;
skipBuffer.writeVInt((int) (curDocPointer - lastSkipDocPointer[level]));
lastSkipDocPointer[level] = curDocPointer;
if (fieldHasPositions) {
skipBuffer.writeVInt((int) (curPosPointer - lastSkipPosPointer[level]));
lastSkipPosPointer[level] = curPosPointer;
skipBuffer.writeVInt(curPosBufferUpto);
if (fieldHasPayloads) {
skipBuffer.writeVInt(curPayloadByteUpto);
}
if (fieldHasOffsets || fieldHasPayloads) {
skipBuffer.writeVInt((int) (curPayPointer - lastSkipPayPointer[level]));
lastSkipPayPointer[level] = curPayPointer;
}
}
}
}

View File

@ -154,20 +154,20 @@ its title, url, or an identifier to access a database. The set of stored fields
returned for each hit when searching. This is keyed by document number.
</li>
<li>
{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term dictionary}.
{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term dictionary}.
A dictionary containing all of the terms used in all of the
indexed fields of all of the documents. The dictionary also contains the number
of documents which contain the term, and pointers to the term's frequency and
proximity data.
</li>
<li>
{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term Frequency data}.
{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Frequency data}.
For each term in the dictionary, the numbers of all the
documents that contain that term, and the frequency of the term in that
document, unless frequencies are omitted (IndexOptions.DOCS_ONLY)
</li>
<li>
{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term Proximity data}.
{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Proximity data}.
For each term in the dictionary, the positions that the
term occurs in each document. Note that this will not exist if all fields in
all documents omit position data.
@ -264,27 +264,27 @@ systems that frequently run out of file handles.</td>
<td>The stored fields for documents</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term Dictionary}</td>
<td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Dictionary}</td>
<td>.tim</td>
<td>The term dictionary, stores term info</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term Index}</td>
<td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Index}</td>
<td>.tip</td>
<td>The index into the Term Dictionary</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Frequencies}</td>
<td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Frequencies}</td>
<td>.doc</td>
<td>Contains the list of docs which contain each term along with frequency</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Positions}</td>
<td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Positions}</td>
<td>.pos</td>
<td>Stores position information about where a term occurs in the index</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Payloads}</td>
<td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Payloads}</td>
<td>.pay</td>
<td>Stores additional per-position metadata information such as character offsets and user payloads</td>
</tr>

View File

@ -13,4 +13,4 @@
# See the License for the specific language governing permissions and
# limitations under the License.
org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat
org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat

View File

@ -1,4 +1,4 @@
package org.apache.lucene.codecs.lucene41;
package org.apache.lucene.codecs.lucene50;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -34,7 +34,7 @@ import org.apache.lucene.util.TestUtil;
* Tests BlockPostingsFormat
*/
public class TestBlockPostingsFormat extends BasePostingsFormatTestCase {
private final Codec codec = TestUtil.alwaysPostingsFormat(new Lucene41PostingsFormat());
private final Codec codec = TestUtil.alwaysPostingsFormat(new Lucene50PostingsFormat());
@Override
protected Codec getCodec() {

View File

@ -1,4 +1,4 @@
package org.apache.lucene.codecs.lucene41;
package org.apache.lucene.codecs.lucene50;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -45,7 +45,7 @@ public class TestBlockPostingsFormat2 extends LuceneTestCase {
super.setUp();
dir = newFSDirectory(createTempDir("testDFBlockSize"));
IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()));
iwc.setCodec(TestUtil.alwaysPostingsFormat(new Lucene41PostingsFormat()));
iwc.setCodec(TestUtil.alwaysPostingsFormat(new Lucene50PostingsFormat()));
iw = new RandomIndexWriter(random(), dir, iwc);
iw.setDoRandomForceMerge(false); // we will ourselves
}
@ -55,7 +55,7 @@ public class TestBlockPostingsFormat2 extends LuceneTestCase {
iw.close();
TestUtil.checkIndex(dir); // for some extra coverage, checkIndex before we forceMerge
IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()));
iwc.setCodec(TestUtil.alwaysPostingsFormat(new Lucene41PostingsFormat()));
iwc.setCodec(TestUtil.alwaysPostingsFormat(new Lucene50PostingsFormat()));
iwc.setOpenMode(OpenMode.APPEND);
IndexWriter iw = new IndexWriter(dir, iwc);
iw.forceMerge(1);
@ -82,7 +82,7 @@ public class TestBlockPostingsFormat2 extends LuceneTestCase {
/** tests terms with df = blocksize */
public void testDFBlockSize() throws Exception {
Document doc = newDocument();
for (int i = 0; i < Lucene41PostingsFormat.BLOCK_SIZE; i++) {
for (int i = 0; i < Lucene50PostingsFormat.BLOCK_SIZE; i++) {
for (Field f : doc.getFields()) {
f.setStringValue(f.name() + " " + f.name() + "_2");
}
@ -93,7 +93,7 @@ public class TestBlockPostingsFormat2 extends LuceneTestCase {
/** tests terms with df % blocksize = 0 */
public void testDFBlockSizeMultiple() throws Exception {
Document doc = newDocument();
for (int i = 0; i < Lucene41PostingsFormat.BLOCK_SIZE * 16; i++) {
for (int i = 0; i < Lucene50PostingsFormat.BLOCK_SIZE * 16; i++) {
for (Field f : doc.getFields()) {
f.setStringValue(f.name() + " " + f.name() + "_2");
}
@ -104,7 +104,7 @@ public class TestBlockPostingsFormat2 extends LuceneTestCase {
/** tests terms with ttf = blocksize */
public void testTTFBlockSize() throws Exception {
Document doc = newDocument();
for (int i = 0; i < Lucene41PostingsFormat.BLOCK_SIZE/2; i++) {
for (int i = 0; i < Lucene50PostingsFormat.BLOCK_SIZE/2; i++) {
for (Field f : doc.getFields()) {
f.setStringValue(f.name() + " " + f.name() + " " + f.name() + "_2 " + f.name() + "_2");
}
@ -115,7 +115,7 @@ public class TestBlockPostingsFormat2 extends LuceneTestCase {
/** tests terms with ttf % blocksize = 0 */
public void testTTFBlockSizeMultiple() throws Exception {
Document doc = newDocument();
for (int i = 0; i < Lucene41PostingsFormat.BLOCK_SIZE/2; i++) {
for (int i = 0; i < Lucene50PostingsFormat.BLOCK_SIZE/2; i++) {
for (Field f : doc.getFields()) {
String proto = (f.name() + " " + f.name() + " " + f.name() + " " + f.name() + " "
+ f.name() + "_2 " + f.name() + "_2 " + f.name() + "_2 " + f.name() + "_2");

View File

@ -1,4 +1,4 @@
package org.apache.lucene.codecs.lucene41;
package org.apache.lucene.codecs.lucene50;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -61,7 +61,7 @@ import org.apache.lucene.util.automaton.RegExp;
* Tests partial enumeration (only pulling a subset of the indexed data)
*/
public class TestBlockPostingsFormat3 extends LuceneTestCase {
static final int MAXDOC = Lucene41PostingsFormat.BLOCK_SIZE * 20;
static final int MAXDOC = Lucene50PostingsFormat.BLOCK_SIZE * 20;
// creates 8 fields with different options and does "duels" of fields against each other
public void test() throws Exception {
@ -82,7 +82,7 @@ public class TestBlockPostingsFormat3 extends LuceneTestCase {
}
};
IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
iwc.setCodec(TestUtil.alwaysPostingsFormat(new Lucene41PostingsFormat()));
iwc.setCodec(TestUtil.alwaysPostingsFormat(new Lucene50PostingsFormat()));
// TODO we could actually add more fields implemented with different PFs
// or, just put this test into the usual rotation?
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
@ -137,7 +137,7 @@ public class TestBlockPostingsFormat3 extends LuceneTestCase {
verify(dir);
TestUtil.checkIndex(dir); // for some extra coverage, checkIndex before we forceMerge
iwc = newIndexWriterConfig(analyzer);
iwc.setCodec(TestUtil.alwaysPostingsFormat(new Lucene41PostingsFormat()));
iwc.setCodec(TestUtil.alwaysPostingsFormat(new Lucene50PostingsFormat()));
iwc.setOpenMode(OpenMode.APPEND);
IndexWriter iw2 = new IndexWriter(dir, iwc);
iw2.forceMerge(1);

View File

@ -1,4 +1,4 @@
package org.apache.lucene.codecs.lucene41;
package org.apache.lucene.codecs.lucene50;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -17,9 +17,9 @@ package org.apache.lucene.codecs.lucene41;
* limitations under the License.
*/
import static org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat.BLOCK_SIZE;
import static org.apache.lucene.codecs.lucene41.ForUtil.MAX_DATA_SIZE;
import static org.apache.lucene.codecs.lucene41.ForUtil.MAX_ENCODED_SIZE;
import static org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat.BLOCK_SIZE;
import static org.apache.lucene.codecs.lucene50.ForUtil.MAX_DATA_SIZE;
import static org.apache.lucene.codecs.lucene50.ForUtil.MAX_ENCODED_SIZE;
import java.io.IOException;
import java.util.Arrays;
@ -89,6 +89,8 @@ public class TestForUtil extends LuceneTestCase {
assertEquals(endPointer, in.getFilePointer());
in.close();
}
d.close();
}
}

View File

@ -26,8 +26,6 @@ import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.asserting.AssertingCodec;
import org.apache.lucene.codecs.blocktreeords.Ords41PostingsFormat;
import org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.SortedSetDocValuesField;
@ -121,17 +119,7 @@ public class TestLucene50DocValuesFormat extends BaseCompressingDocValuesFormatT
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
conf.setMergeScheduler(new SerialMergeScheduler());
// set to duel against a codec which has ordinals:
final PostingsFormat pf;
switch (random().nextInt(2)) {
case 0: pf = new Lucene41WithOrds();
break;
case 1: pf = new Ords41PostingsFormat();
break;
// TODO: these don't actually support ords!
//case 2: pf = new FSTOrdPostingsFormat();
// break;
default: throw new AssertionError();
}
final PostingsFormat pf = TestUtil.getPostingsFormatWithOrds(random());
final DocValuesFormat dv = new Lucene50DocValuesFormat();
conf.setCodec(new AssertingCodec() {
@Override

View File

@ -22,7 +22,7 @@ import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.asserting.AssertingCodec;
import org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapFixedInterval;
import org.apache.lucene.codecs.blockterms.LuceneVarGapFixedInterval;
import org.apache.lucene.codecs.memory.MemoryPostingsFormat;
import org.apache.lucene.codecs.simpletext.SimpleTextPostingsFormat;
import org.apache.lucene.document.Document;
@ -287,9 +287,9 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
if ("id".equals(field)) {
return new Lucene41VarGapFixedInterval(1);
return new LuceneVarGapFixedInterval(1);
} else if ("date".equals(field)) {
return new Lucene41VarGapFixedInterval(2);
return new LuceneVarGapFixedInterval(2);
} else {
return super.getPostingsFormatForField(field);
}

View File

@ -27,7 +27,6 @@ import java.util.Set;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.IntField;
@ -129,7 +128,7 @@ public class TestDocTermOrds extends LuceneTestCase {
// Sometimes swap in codec that impls ord():
if (random().nextInt(10) == 7) {
// Make sure terms index has ords:
Codec codec = TestUtil.alwaysPostingsFormat(PostingsFormat.forName("Lucene41WithOrds"));
Codec codec = TestUtil.alwaysPostingsFormat(TestUtil.getPostingsFormatWithOrds(random()));
conf.setCodec(codec);
}
@ -226,7 +225,7 @@ public class TestDocTermOrds extends LuceneTestCase {
// Sometimes swap in codec that impls ord():
if (random().nextInt(10) == 7) {
Codec codec = TestUtil.alwaysPostingsFormat(PostingsFormat.forName("Lucene41WithOrds"));
Codec codec = TestUtil.alwaysPostingsFormat(TestUtil.getPostingsFormatWithOrds(random()));
conf.setCodec(codec);
}

View File

@ -81,7 +81,7 @@ public class IDVersionPostingsFormat extends PostingsFormat {
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase postingsWriter = new IDVersionPostingsWriter(state);
PostingsWriterBase postingsWriter = new IDVersionPostingsWriter(state.liveDocs);
boolean success = false;
try {
FieldsConsumer ret = new VersionBlockTreeTermsWriter(state,
@ -99,7 +99,7 @@ public class IDVersionPostingsFormat extends PostingsFormat {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postingsReader = new IDVersionPostingsReader(state);
PostingsReaderBase postingsReader = new IDVersionPostingsReader();
boolean success = false;
try {
FieldsProducer ret = new VersionBlockTreeTermsReader(postingsReader, state);

View File

@ -33,14 +33,9 @@ import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Bits;
final class IDVersionPostingsReader extends PostingsReaderBase {
final SegmentReadState state;
public IDVersionPostingsReader(SegmentReadState state) {
this.state = state;
}
@Override
public void init(IndexInput termsIn) throws IOException {
public void init(IndexInput termsIn, SegmentReadState state) throws IOException {
// Make sure we are talking to the matching postings writer
CodecUtil.checkSegmentHeader(termsIn,
IDVersionPostingsWriter.TERMS_CODEC,

View File

@ -26,6 +26,7 @@ import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
final class IDVersionPostingsWriter extends PushPostingsWriterBase {
@ -43,10 +44,10 @@ final class IDVersionPostingsWriter extends PushPostingsWriterBase {
private int lastPosition;
private long lastVersion;
private final SegmentWriteState state;
private final Bits liveDocs;
public IDVersionPostingsWriter(SegmentWriteState state) {
this.state = state;
public IDVersionPostingsWriter(Bits liveDocs) {
this.liveDocs = liveDocs;
}
@Override
@ -55,7 +56,7 @@ final class IDVersionPostingsWriter extends PushPostingsWriterBase {
}
@Override
public void init(IndexOutput termsOut) throws IOException {
public void init(IndexOutput termsOut, SegmentWriteState state) throws IOException {
CodecUtil.writeSegmentHeader(termsOut, TERMS_CODEC, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
}
@ -82,7 +83,7 @@ final class IDVersionPostingsWriter extends PushPostingsWriterBase {
@Override
public void startDoc(int docID, int termDocFreq) throws IOException {
// TODO: LUCENE-5693: we don't need this check if we fix IW to not send deleted docs to us on flush:
if (state.liveDocs != null && state.liveDocs.get(docID) == false) {
if (liveDocs != null && liveDocs.get(docID) == false) {
return;
}
if (lastDocID != -1) {

View File

@ -94,7 +94,7 @@ public final class VersionBlockTreeTermsReader extends FieldsProducer {
CodecUtil.checksumEntireFile(indexIn);
// Have PostingsReader init itself
postingsReader.init(in);
postingsReader.init(in, state);
// NOTE: data file is too costly to verify checksum against all the bytes on open,
// but for now we at least verify proper structure of the checksum footer: which looks

View File

@ -212,7 +212,7 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
// System.out.println("BTW.init seg=" + state.segmentName);
postingsWriter.init(out); // have consumer write its format/header
postingsWriter.init(out, state); // have consumer write its format/header
success = true;
} finally {
if (!success) {

View File

@ -1,4 +1,4 @@
package org.apache.lucene.codecs.lucene41ords;
package org.apache.lucene.codecs.blockterms;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -30,9 +30,9 @@ import org.apache.lucene.codecs.blockterms.FixedGapTermsIndexReader;
import org.apache.lucene.codecs.blockterms.FixedGapTermsIndexWriter;
import org.apache.lucene.codecs.blockterms.TermsIndexReaderBase;
import org.apache.lucene.codecs.blockterms.TermsIndexWriterBase;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat; // javadocs
import org.apache.lucene.codecs.lucene41.Lucene41PostingsReader;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat; // javadocs
import org.apache.lucene.codecs.lucene50.Lucene50PostingsReader;
import org.apache.lucene.codecs.lucene50.Lucene50PostingsWriter;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
@ -40,24 +40,24 @@ import org.apache.lucene.index.SegmentWriteState;
// any PostingsBaseFormat and make it ord-able...
/**
* Customized version of {@link Lucene41PostingsFormat} that uses
* Customized version of {@link Lucene50PostingsFormat} that uses
* {@link FixedGapTermsIndexWriter}.
*/
public final class Lucene41WithOrds extends PostingsFormat {
public final class LuceneFixedGap extends PostingsFormat {
final int termIndexInterval;
public Lucene41WithOrds() {
public LuceneFixedGap() {
this(FixedGapTermsIndexWriter.DEFAULT_TERM_INDEX_INTERVAL);
}
public Lucene41WithOrds(int termIndexInterval) {
super("Lucene41WithOrds");
public LuceneFixedGap(int termIndexInterval) {
super("LuceneFixedGap");
this.termIndexInterval = termIndexInterval;
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase docs = new Lucene41PostingsWriter(state);
PostingsWriterBase docs = new Lucene50PostingsWriter(state);
// TODO: should we make the terms index more easily
// pluggable? Ie so that this codec would record which
@ -94,7 +94,7 @@ public final class Lucene41WithOrds extends PostingsFormat {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postings = new Lucene41PostingsReader(state.directory, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
PostingsReaderBase postings = new Lucene50PostingsReader(state);
TermsIndexReaderBase indexReader;
boolean success = false;

View File

@ -1,4 +1,4 @@
package org.apache.lucene.codecs.lucene41vargap;
package org.apache.lucene.codecs.blockterms;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -31,9 +31,9 @@ import org.apache.lucene.codecs.blockterms.TermsIndexReaderBase;
import org.apache.lucene.codecs.blockterms.TermsIndexWriterBase;
import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexReader;
import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexWriter;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat; // javadocs
import org.apache.lucene.codecs.lucene41.Lucene41PostingsReader;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat; // javadocs
import org.apache.lucene.codecs.lucene50.Lucene50PostingsReader;
import org.apache.lucene.codecs.lucene50.Lucene50PostingsWriter;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
@ -41,27 +41,27 @@ import org.apache.lucene.index.SegmentWriteState;
// any PostingsBaseFormat and make it ord-able...
/**
* Customized version of {@link Lucene41PostingsFormat} that uses
* Customized version of {@link Lucene50PostingsFormat} that uses
* {@link VariableGapTermsIndexWriter} with a fixed interval, but
* forcing high docfreq terms to be indexed terms.
*/
public final class Lucene41VarGapDocFreqInterval extends PostingsFormat {
public final class LuceneVarGapDocFreqInterval extends PostingsFormat {
final int termIndexInterval;
final int docFreqThreshold;
public Lucene41VarGapDocFreqInterval() {
public LuceneVarGapDocFreqInterval() {
this(1000000, FixedGapTermsIndexWriter.DEFAULT_TERM_INDEX_INTERVAL);
}
public Lucene41VarGapDocFreqInterval(int docFreqThreshold, int termIndexInterval) {
super("Lucene41VarGapFixedInterval");
public LuceneVarGapDocFreqInterval(int docFreqThreshold, int termIndexInterval) {
super("LuceneVarGapDocFreqInterval");
this.termIndexInterval = termIndexInterval;
this.docFreqThreshold = docFreqThreshold;
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase docs = new Lucene41PostingsWriter(state);
PostingsWriterBase docs = new Lucene50PostingsWriter(state);
// TODO: should we make the terms index more easily
// pluggable? Ie so that this codec would record which
@ -98,7 +98,7 @@ public final class Lucene41VarGapDocFreqInterval extends PostingsFormat {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postings = new Lucene41PostingsReader(state.directory, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
PostingsReaderBase postings = new Lucene50PostingsReader(state);
TermsIndexReaderBase indexReader;
boolean success = false;

View File

@ -1,4 +1,4 @@
package org.apache.lucene.codecs.lucene41vargap;
package org.apache.lucene.codecs.blockterms;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -31,9 +31,9 @@ import org.apache.lucene.codecs.blockterms.TermsIndexReaderBase;
import org.apache.lucene.codecs.blockterms.TermsIndexWriterBase;
import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexReader;
import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexWriter;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat; // javadocs
import org.apache.lucene.codecs.lucene41.Lucene41PostingsReader;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat; // javadocs
import org.apache.lucene.codecs.lucene50.Lucene50PostingsReader;
import org.apache.lucene.codecs.lucene50.Lucene50PostingsWriter;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
@ -41,24 +41,24 @@ import org.apache.lucene.index.SegmentWriteState;
// any PostingsBaseFormat and make it ord-able...
/**
* Customized version of {@link Lucene41PostingsFormat} that uses
* Customized version of {@link Lucene50PostingsFormat} that uses
* {@link VariableGapTermsIndexWriter} with a fixed interval.
*/
public final class Lucene41VarGapFixedInterval extends PostingsFormat {
public final class LuceneVarGapFixedInterval extends PostingsFormat {
final int termIndexInterval;
public Lucene41VarGapFixedInterval() {
public LuceneVarGapFixedInterval() {
this(FixedGapTermsIndexWriter.DEFAULT_TERM_INDEX_INTERVAL);
}
public Lucene41VarGapFixedInterval(int termIndexInterval) {
super("Lucene41VarGapFixedInterval");
public LuceneVarGapFixedInterval(int termIndexInterval) {
super("LuceneVarGapFixedInterval");
this.termIndexInterval = termIndexInterval;
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase docs = new Lucene41PostingsWriter(state);
PostingsWriterBase docs = new Lucene50PostingsWriter(state);
// TODO: should we make the terms index more easily
// pluggable? Ie so that this codec would record which
@ -95,7 +95,7 @@ public final class Lucene41VarGapFixedInterval extends PostingsFormat {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postings = new Lucene41PostingsReader(state.directory, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
PostingsReaderBase postings = new Lucene50PostingsReader(state);
TermsIndexReaderBase indexReader;
boolean success = false;

View File

@ -20,6 +20,6 @@
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
</head>
<body>
Codecs for testing that support {@link org.apache.lucene.codecs.blockterms.VariableGapTermsIndexReader}
Codecs for testing that support {@link org.apache.lucene.codecs.blockterms}
</body>
</html>

View File

@ -1,25 +0,0 @@
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
<!--
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.
-->
<html>
<head>
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
</head>
<body>
Codec for testing that supports {@link org.apache.lucene.index.TermsEnum#ord()}
</body>
</html>

View File

@ -38,8 +38,8 @@ import org.apache.lucene.codecs.blocktree.BlockTreeTermsReader;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter;
import org.apache.lucene.codecs.blocktreeords.OrdsBlockTreeTermsReader;
import org.apache.lucene.codecs.blocktreeords.OrdsBlockTreeTermsWriter;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsReader;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
import org.apache.lucene.codecs.lucene50.Lucene50PostingsReader;
import org.apache.lucene.codecs.lucene50.Lucene50PostingsWriter;
import org.apache.lucene.codecs.memory.FSTOrdTermsReader;
import org.apache.lucene.codecs.memory.FSTOrdTermsWriter;
import org.apache.lucene.codecs.memory.FSTTermsReader;
@ -92,7 +92,9 @@ public final class MockRandomPostingsFormat extends PostingsFormat {
}
// we pull this before the seed intentionally: because its not consumed at runtime
// (the skipInterval is written into postings header)
// (the skipInterval is written into postings header).
// NOTE: Currently not passed to postings writer.
// before, it was being passed in wrongly as acceptableOverhead!
int skipInterval = TestUtil.nextInt(seedRandom, minSkipInterval, 10);
if (LuceneTestCase.VERBOSE) {
@ -117,7 +119,7 @@ public final class MockRandomPostingsFormat extends PostingsFormat {
random.nextInt(); // consume a random for buffersize
PostingsWriterBase postingsWriter = new Lucene41PostingsWriter(state, skipInterval);
PostingsWriterBase postingsWriter = new Lucene50PostingsWriter(state);
final FieldsConsumer fields;
final int t1 = random.nextInt(5);
@ -280,7 +282,7 @@ public final class MockRandomPostingsFormat extends PostingsFormat {
System.out.println("MockRandomCodec: readBufferSize=" + readBufferSize);
}
PostingsReaderBase postingsReader = new Lucene41PostingsReader(state.directory, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
PostingsReaderBase postingsReader = new Lucene50PostingsReader(state);
final FieldsProducer fields;
final int t1 = random.nextInt(5);
@ -312,12 +314,7 @@ public final class MockRandomPostingsFormat extends PostingsFormat {
boolean success = false;
try {
fields = new BlockTreeTermsReader(state.directory,
state.fieldInfos,
state.segmentInfo,
postingsReader,
state.context,
state.segmentSuffix);
fields = new BlockTreeTermsReader(postingsReader, state);
success = true;
} finally {
if (!success) {

View File

@ -32,11 +32,11 @@ import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.asserting.AssertingCodec;
import org.apache.lucene.codecs.asserting.AssertingDocValuesFormat;
import org.apache.lucene.codecs.asserting.AssertingPostingsFormat;
import org.apache.lucene.codecs.blocktreeords.Ords41PostingsFormat;
import org.apache.lucene.codecs.blockterms.LuceneFixedGap;
import org.apache.lucene.codecs.blockterms.LuceneVarGapDocFreqInterval;
import org.apache.lucene.codecs.blockterms.LuceneVarGapFixedInterval;
import org.apache.lucene.codecs.blocktreeords.BlockTreeOrdsPostingsFormat;
import org.apache.lucene.codecs.bloom.TestBloomFilteredLucenePostings;
import org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds;
import org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapDocFreqInterval;
import org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapFixedInterval;
import org.apache.lucene.codecs.memory.DirectPostingsFormat;
import org.apache.lucene.codecs.memory.FSTOrdPostingsFormat;
import org.apache.lucene.codecs.memory.FSTPostingsFormat;
@ -127,15 +127,15 @@ public class RandomCodec extends AssertingCodec {
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 TestBloomFilteredLucene41Postings to be constructed
//TODO as a PostingsFormat which wraps others, we should allow TestBloomFilteredLucenePostings to be constructed
//with a choice of concrete PostingsFormats. Maybe useful to have a generic means of marking and dealing
//with such "wrapper" classes?
new TestBloomFilteredLucenePostings(),
new MockRandomPostingsFormat(random),
new Ords41PostingsFormat(minItemsPerBlock, maxItemsPerBlock),
new Lucene41WithOrds(TestUtil.nextInt(random, 1, 1000)),
new Lucene41VarGapFixedInterval(TestUtil.nextInt(random, 1, 1000)),
new Lucene41VarGapDocFreqInterval(TestUtil.nextInt(random, 1, 100), TestUtil.nextInt(random, 1, 1000)),
new BlockTreeOrdsPostingsFormat(minItemsPerBlock, maxItemsPerBlock),
new LuceneFixedGap(TestUtil.nextInt(random, 1, 1000)),
new LuceneVarGapFixedInterval(TestUtil.nextInt(random, 1, 1000)),
new LuceneVarGapDocFreqInterval(TestUtil.nextInt(random, 1, 100), TestUtil.nextInt(random, 1, 1000)),
new SimpleTextPostingsFormat(),
new AssertingPostingsFormat(),
new MemoryPostingsFormat(true, random.nextFloat()),

View File

@ -45,9 +45,11 @@ import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.asserting.AssertingCodec;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
import org.apache.lucene.codecs.blockterms.LuceneFixedGap;
import org.apache.lucene.codecs.blocktreeords.BlockTreeOrdsPostingsFormat;
import org.apache.lucene.codecs.lucene50.Lucene50Codec;
import org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat;
import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
import org.apache.lucene.document.BinaryDocValuesField;
@ -746,7 +748,7 @@ public final class TestUtil {
* Returns the actual default postings format (e.g. LuceneMNPostingsFormat for this version of Lucene.
*/
public static PostingsFormat getDefaultPostingsFormat() {
return new Lucene41PostingsFormat();
return new Lucene50PostingsFormat();
}
/**
@ -754,7 +756,18 @@ public final class TestUtil {
* @lucene.internal this may disappear at any time
*/
public static PostingsFormat getDefaultPostingsFormat(int minItemsPerBlock, int maxItemsPerBlock) {
return new Lucene41PostingsFormat(minItemsPerBlock, maxItemsPerBlock);
return new Lucene50PostingsFormat(minItemsPerBlock, maxItemsPerBlock);
}
/** Returns a random postings format that supports term ordinals */
public static PostingsFormat getPostingsFormatWithOrds(Random r) {
switch (r.nextInt(2)) {
case 0: return new LuceneFixedGap();
case 1: return new BlockTreeOrdsPostingsFormat();
// TODO: these don't actually support ords!
//case 2: return new FSTOrdPostingsFormat();
default: throw new AssertionError();
}
}
/**

View File

@ -15,8 +15,8 @@
org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat
org.apache.lucene.codecs.ramonly.RAMOnlyPostingsFormat
org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds
org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapFixedInterval
org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapDocFreqInterval
org.apache.lucene.codecs.blockterms.LuceneFixedGap
org.apache.lucene.codecs.blockterms.LuceneVarGapFixedInterval
org.apache.lucene.codecs.blockterms.LuceneVarGapDocFreqInterval
org.apache.lucene.codecs.bloom.TestBloomFilteredLucenePostings
org.apache.lucene.codecs.asserting.AssertingPostingsFormat

View File

@ -19,7 +19,7 @@
<types>
<fieldType name="string_direct" class="solr.StrField" postingsFormat="Direct"/>
<fieldType name="string_simpletext" class="solr.StrField" postingsFormat="SimpleText"/>
<fieldType name="string_standard" class="solr.StrField" postingsFormat="Lucene41"/>
<fieldType name="string_standard" class="solr.StrField" postingsFormat="Lucene50"/>
<fieldType name="string_disk" class="solr.StrField" docValuesFormat="Lucene50" />
<fieldType name="string_memory" class="solr.StrField" docValuesFormat="Memory" />