LUCENE-9843: Remove compression option on default codec's docvalues

This commit is contained in:
Robert Muir 2021-05-06 17:07:41 -04:00
parent e2788336d4
commit a7a02519f0
No known key found for this signature in database
GPG Key ID: 817AE1DD322D7ECA
10 changed files with 146 additions and 899 deletions

View File

@ -114,6 +114,8 @@ API Changes
* LUCENE-9948: No longer require the user to specify whether-or-not a field is multi-valued in
LongValueFacetCounts (detect automatically based on what is indexed). (Greg Miller)
* LUCENE-9843: Remove compression option on default codec's docvalues. (Jack Conradson)
Improvements
* LUCENE-9687: Hunspell support improvements: add API for spell-checking and suggestions, support compound words,

View File

@ -47,18 +47,14 @@ public class Lucene90Codec extends Codec {
/** Configuration option for the codec. */
public enum Mode {
/** Trade compression ratio for retrieval speed. */
BEST_SPEED(Lucene90StoredFieldsFormat.Mode.BEST_SPEED, Lucene90DocValuesFormat.Mode.BEST_SPEED),
BEST_SPEED(Lucene90StoredFieldsFormat.Mode.BEST_SPEED),
/** Trade retrieval speed for compression ratio. */
BEST_COMPRESSION(
Lucene90StoredFieldsFormat.Mode.BEST_COMPRESSION,
Lucene90DocValuesFormat.Mode.BEST_COMPRESSION);
BEST_COMPRESSION(Lucene90StoredFieldsFormat.Mode.BEST_COMPRESSION);
private final Lucene90StoredFieldsFormat.Mode storedMode;
private final Lucene90DocValuesFormat.Mode dvMode;
private Mode(Lucene90StoredFieldsFormat.Mode storedMode, Lucene90DocValuesFormat.Mode dvMode) {
private Mode(Lucene90StoredFieldsFormat.Mode storedMode) {
this.storedMode = Objects.requireNonNull(storedMode);
this.dvMode = Objects.requireNonNull(dvMode);
}
}
@ -110,7 +106,7 @@ public class Lucene90Codec extends Codec {
this.storedFieldsFormat =
new Lucene90StoredFieldsFormat(Objects.requireNonNull(mode).storedMode);
this.defaultFormat = new Lucene90PostingsFormat();
this.defaultDVFormat = new Lucene90DocValuesFormat(mode.dvMode);
this.defaultDVFormat = new Lucene90DocValuesFormat();
}
@Override

View File

@ -20,7 +20,6 @@ import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.DIRECT_M
import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.NUMERIC_BLOCK_SHIFT;
import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.NUMERIC_BLOCK_SIZE;
import java.io.Closeable;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashMap;
@ -31,7 +30,6 @@ import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.EmptyDocValuesProducer;
import org.apache.lucene.index.FieldInfo;
@ -46,9 +44,6 @@ import org.apache.lucene.search.SortedSetSelector;
import org.apache.lucene.store.ByteArrayDataOutput;
import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.store.ByteBuffersIndexOutput;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
@ -57,17 +52,14 @@ import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.MathUtil;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.compress.LZ4;
import org.apache.lucene.util.compress.LZ4.FastCompressionHashTable;
import org.apache.lucene.util.packed.DirectMonotonicWriter;
import org.apache.lucene.util.packed.DirectWriter;
/** writer for {@link Lucene90DocValuesFormat} */
final class Lucene90DocValuesConsumer extends DocValuesConsumer {
final Lucene90DocValuesFormat.Mode mode;
IndexOutput data, meta;
final int maxDoc;
private final SegmentWriteState state;
private byte[] termsDictBuffer;
/** expert: Creates a new writer */
@ -76,16 +68,11 @@ final class Lucene90DocValuesConsumer extends DocValuesConsumer {
String dataCodec,
String dataExtension,
String metaCodec,
String metaExtension,
Lucene90DocValuesFormat.Mode mode)
String metaExtension)
throws IOException {
this.mode = mode;
if (Lucene90DocValuesFormat.Mode.BEST_COMPRESSION == this.mode) {
this.termsDictBuffer = new byte[1 << 14];
}
this.termsDictBuffer = new byte[1 << 14];
boolean success = false;
try {
this.state = state;
String dataName =
IndexFileNames.segmentFileName(
state.segmentInfo.name, state.segmentSuffix, dataExtension);
@ -402,167 +389,11 @@ final class Lucene90DocValuesConsumer extends DocValuesConsumer {
}
}
class CompressedBinaryBlockWriter implements Closeable {
final FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();
int uncompressedBlockLength = 0;
int maxUncompressedBlockLength = 0;
int numDocsInCurrentBlock = 0;
final int[] docLengths = new int[Lucene90DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK];
byte[] block = BytesRef.EMPTY_BYTES;
int totalChunks = 0;
long maxPointer = 0;
final long blockAddressesStart;
private final IndexOutput tempBinaryOffsets;
public CompressedBinaryBlockWriter() throws IOException {
tempBinaryOffsets =
state.directory.createTempOutput(
state.segmentInfo.name, "binary_pointers", state.context);
boolean success = false;
try {
CodecUtil.writeHeader(
tempBinaryOffsets,
Lucene90DocValuesFormat.META_CODEC + "FilePointers",
Lucene90DocValuesFormat.VERSION_CURRENT);
blockAddressesStart = data.getFilePointer();
success = true;
} finally {
if (success == false) {
IOUtils.closeWhileHandlingException(this); // self-close because constructor caller can't
}
}
}
void addDoc(int doc, BytesRef v) throws IOException {
docLengths[numDocsInCurrentBlock] = v.length;
block = ArrayUtil.grow(block, uncompressedBlockLength + v.length);
System.arraycopy(v.bytes, v.offset, block, uncompressedBlockLength, v.length);
uncompressedBlockLength += v.length;
numDocsInCurrentBlock++;
if (numDocsInCurrentBlock == Lucene90DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK) {
flushData();
}
}
private void flushData() throws IOException {
if (numDocsInCurrentBlock > 0) {
// Write offset to this block to temporary offsets file
totalChunks++;
long thisBlockStartPointer = data.getFilePointer();
// Optimisation - check if all lengths are same
boolean allLengthsSame = true;
for (int i = 1; i < Lucene90DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK; i++) {
if (docLengths[i] != docLengths[i - 1]) {
allLengthsSame = false;
break;
}
}
if (allLengthsSame) {
// Only write one value shifted. Steal a bit to indicate all other lengths are the same
int onlyOneLength = (docLengths[0] << 1) | 1;
data.writeVInt(onlyOneLength);
} else {
for (int i = 0; i < Lucene90DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK; i++) {
if (i == 0) {
// Write first value shifted and steal a bit to indicate other lengths are to follow
int multipleLengths = (docLengths[0] << 1);
data.writeVInt(multipleLengths);
} else {
data.writeVInt(docLengths[i]);
}
}
}
maxUncompressedBlockLength = Math.max(maxUncompressedBlockLength, uncompressedBlockLength);
LZ4.compress(block, 0, uncompressedBlockLength, data, ht);
numDocsInCurrentBlock = 0;
// Ensure initialized with zeroes because full array is always written
Arrays.fill(docLengths, 0);
uncompressedBlockLength = 0;
maxPointer = data.getFilePointer();
tempBinaryOffsets.writeVLong(maxPointer - thisBlockStartPointer);
}
}
void writeMetaData() throws IOException {
if (totalChunks == 0) {
return;
}
long startDMW = data.getFilePointer();
meta.writeLong(startDMW);
meta.writeVInt(totalChunks);
meta.writeVInt(Lucene90DocValuesFormat.BINARY_BLOCK_SHIFT);
meta.writeVInt(maxUncompressedBlockLength);
meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
CodecUtil.writeFooter(tempBinaryOffsets);
IOUtils.close(tempBinaryOffsets);
// write the compressed block offsets info to the meta file by reading from temp file
try (ChecksumIndexInput filePointersIn =
state.directory.openChecksumInput(tempBinaryOffsets.getName(), IOContext.READONCE)) {
CodecUtil.checkHeader(
filePointersIn,
Lucene90DocValuesFormat.META_CODEC + "FilePointers",
Lucene90DocValuesFormat.VERSION_CURRENT,
Lucene90DocValuesFormat.VERSION_CURRENT);
Throwable priorE = null;
try {
final DirectMonotonicWriter filePointers =
DirectMonotonicWriter.getInstance(
meta, data, totalChunks, DIRECT_MONOTONIC_BLOCK_SHIFT);
long fp = blockAddressesStart;
for (int i = 0; i < totalChunks; ++i) {
filePointers.add(fp);
fp += filePointersIn.readVLong();
}
if (maxPointer < fp) {
throw new CorruptIndexException(
"File pointers don't add up (" + fp + " vs expected " + maxPointer + ")",
filePointersIn);
}
filePointers.finish();
} catch (Throwable e) {
priorE = e;
} finally {
CodecUtil.checkFooter(filePointersIn, priorE);
}
}
// Write the length of the DMW block in the data
meta.writeLong(data.getFilePointer() - startDMW);
}
@Override
public void close() throws IOException {
if (tempBinaryOffsets != null) {
IOUtils.close(tempBinaryOffsets);
state.directory.deleteFile(tempBinaryOffsets.getName());
}
}
}
@Override
public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
field.putAttribute(Lucene90DocValuesFormat.MODE_KEY, mode.name());
meta.writeInt(field.number);
meta.writeByte(Lucene90DocValuesFormat.BINARY);
switch (mode) {
case BEST_SPEED:
doAddUncompressedBinaryField(field, valuesProducer);
break;
case BEST_COMPRESSION:
doAddCompressedBinaryField(field, valuesProducer);
break;
default:
throw new AssertionError();
}
}
private void doAddUncompressedBinaryField(FieldInfo field, DocValuesProducer valuesProducer)
throws IOException {
BinaryDocValues values = valuesProducer.getBinary(field);
long start = data.getFilePointer();
meta.writeLong(start); // dataOffset
@ -626,59 +457,6 @@ final class Lucene90DocValuesConsumer extends DocValuesConsumer {
}
}
private void doAddCompressedBinaryField(FieldInfo field, DocValuesProducer valuesProducer)
throws IOException {
try (CompressedBinaryBlockWriter blockWriter = new CompressedBinaryBlockWriter()) {
BinaryDocValues values = valuesProducer.getBinary(field);
long start = data.getFilePointer();
meta.writeLong(start); // dataOffset
int numDocsWithField = 0;
int minLength = Integer.MAX_VALUE;
int maxLength = 0;
for (int doc = values.nextDoc();
doc != DocIdSetIterator.NO_MORE_DOCS;
doc = values.nextDoc()) {
numDocsWithField++;
BytesRef v = values.binaryValue();
blockWriter.addDoc(doc, v);
int length = v.length;
minLength = Math.min(length, minLength);
maxLength = Math.max(length, maxLength);
}
blockWriter.flushData();
assert numDocsWithField <= maxDoc;
meta.writeLong(data.getFilePointer() - start); // dataLength
if (numDocsWithField == 0) {
meta.writeLong(-2); // docsWithFieldOffset
meta.writeLong(0L); // docsWithFieldLength
meta.writeShort((short) -1); // jumpTableEntryCount
meta.writeByte((byte) -1); // denseRankPower
} else if (numDocsWithField == maxDoc) {
meta.writeLong(-1); // docsWithFieldOffset
meta.writeLong(0L); // docsWithFieldLength
meta.writeShort((short) -1); // jumpTableEntryCount
meta.writeByte((byte) -1); // denseRankPower
} else {
long offset = data.getFilePointer();
meta.writeLong(offset); // docsWithFieldOffset
values = valuesProducer.getBinary(field);
final short jumpTableEntryCount =
IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER);
meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength
meta.writeShort(jumpTableEntryCount);
meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER);
}
meta.writeInt(numDocsWithField);
meta.writeInt(minLength);
meta.writeInt(maxLength);
blockWriter.writeMetaData();
}
}
@Override
public void addSortedField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
meta.writeInt(field.number);
@ -742,21 +520,10 @@ final class Lucene90DocValuesConsumer extends DocValuesConsumer {
private void addTermsDict(SortedSetDocValues values) throws IOException {
final long size = values.getValueCount();
meta.writeVLong(size);
boolean compress =
Lucene90DocValuesFormat.Mode.BEST_COMPRESSION == mode
&& values.getValueCount()
> Lucene90DocValuesFormat.TERMS_DICT_BLOCK_COMPRESSION_THRESHOLD;
int code, blockMask, shift;
if (compress) {
code = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_CODE;
blockMask = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_MASK;
shift = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_SHIFT;
} else {
code = shift = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_SHIFT;
blockMask = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_MASK;
}
meta.writeInt(code);
int blockMask = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_MASK;
int shift = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_SHIFT;
meta.writeInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
ByteBuffersDataOutput addressBuffer = new ByteBuffersDataOutput();
ByteBuffersIndexOutput addressOutput =
@ -772,16 +539,12 @@ final class Lucene90DocValuesConsumer extends DocValuesConsumer {
int maxLength = 0, maxBlockLength = 0;
TermsEnum iterator = values.termsEnum();
LZ4.FastCompressionHashTable ht = null;
ByteArrayDataOutput bufferedOutput = null;
if (compress) {
ht = new LZ4.FastCompressionHashTable();
bufferedOutput = new ByteArrayDataOutput(termsDictBuffer);
}
LZ4.FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();
ByteArrayDataOutput bufferedOutput = new ByteArrayDataOutput(termsDictBuffer);
for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
if ((ord & blockMask) == 0) {
if (compress && bufferedOutput.getPosition() > 0) {
if (bufferedOutput.getPosition() > 0) {
maxBlockLength =
Math.max(maxBlockLength, compressAndGetTermsDictBlockLength(bufferedOutput, ht));
bufferedOutput.reset(termsDictBuffer);
@ -794,40 +557,32 @@ final class Lucene90DocValuesConsumer extends DocValuesConsumer {
final int prefixLength = StringHelper.bytesDifference(previous.get(), term);
final int suffixLength = term.length - prefixLength;
assert suffixLength > 0; // terms are unique
DataOutput blockOutput;
if (compress) {
// Will write (suffixLength + 1 byte + 2 vint) bytes. Grow the buffer in need.
bufferedOutput = maybeGrowBuffer(bufferedOutput, suffixLength + 11);
blockOutput = bufferedOutput;
} else {
blockOutput = data;
}
blockOutput.writeByte(
// Will write (suffixLength + 1 byte + 2 vint) bytes. Grow the buffer in need.
bufferedOutput = maybeGrowBuffer(bufferedOutput, suffixLength + 11);
bufferedOutput.writeByte(
(byte) (Math.min(prefixLength, 15) | (Math.min(15, suffixLength - 1) << 4)));
if (prefixLength >= 15) {
blockOutput.writeVInt(prefixLength - 15);
bufferedOutput.writeVInt(prefixLength - 15);
}
if (suffixLength >= 16) {
blockOutput.writeVInt(suffixLength - 16);
bufferedOutput.writeVInt(suffixLength - 16);
}
blockOutput.writeBytes(term.bytes, term.offset + prefixLength, suffixLength);
bufferedOutput.writeBytes(term.bytes, term.offset + prefixLength, suffixLength);
}
maxLength = Math.max(maxLength, term.length);
previous.copyBytes(term);
++ord;
}
// Compress and write out the last block
if (compress && bufferedOutput.getPosition() > 0) {
if (bufferedOutput.getPosition() > 0) {
maxBlockLength =
Math.max(maxBlockLength, compressAndGetTermsDictBlockLength(bufferedOutput, ht));
}
writer.finish();
meta.writeInt(maxLength);
if (compress) {
// Write one more int for storing max block length. For compressed terms dict only.
meta.writeInt(maxBlockLength);
}
// Write one more int for storing max block length.
meta.writeInt(maxBlockLength);
meta.writeLong(start);
meta.writeLong(data.getFilePointer() - start);
start = data.getFilePointer();

View File

@ -17,7 +17,6 @@
package org.apache.lucene.codecs.lucene90;
import java.io.IOException;
import java.util.Objects;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.DocValuesProducer;
@ -139,34 +138,15 @@ import org.apache.lucene.util.packed.DirectWriter;
*/
public final class Lucene90DocValuesFormat extends DocValuesFormat {
/** Configuration option for doc values. */
public enum Mode {
/** Trade compression ratio for retrieval speed. */
BEST_SPEED,
/** Trade retrieval speed for compression ratio. */
BEST_COMPRESSION
}
/** Attribute key for compression mode. */
public static final String MODE_KEY = Lucene90DocValuesFormat.class.getSimpleName() + ".mode";
private final Mode mode;
/** Default constructor. */
public Lucene90DocValuesFormat() {
this(Mode.BEST_SPEED);
}
/** Constructor */
public Lucene90DocValuesFormat(Mode mode) {
super("Lucene90");
this.mode = Objects.requireNonNull(mode);
}
@Override
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
return new Lucene90DocValuesConsumer(
state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION, mode);
state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
}
@Override
@ -194,21 +174,9 @@ public final class Lucene90DocValuesFormat extends DocValuesFormat {
static final int NUMERIC_BLOCK_SHIFT = 14;
static final int NUMERIC_BLOCK_SIZE = 1 << NUMERIC_BLOCK_SHIFT;
static final int BINARY_BLOCK_SHIFT = 5;
static final int BINARY_DOCS_PER_COMPRESSED_BLOCK = 1 << BINARY_BLOCK_SHIFT;
static final int TERMS_DICT_BLOCK_SHIFT = 4;
static final int TERMS_DICT_BLOCK_SIZE = 1 << TERMS_DICT_BLOCK_SHIFT;
static final int TERMS_DICT_BLOCK_MASK = TERMS_DICT_BLOCK_SIZE - 1;
static final int TERMS_DICT_BLOCK_COMPRESSION_THRESHOLD = 32;
static final int TERMS_DICT_BLOCK_LZ4_SHIFT = 6;
static final int TERMS_DICT_BLOCK_LZ4_SIZE = 1 << TERMS_DICT_BLOCK_LZ4_SHIFT;
static final int TERMS_DICT_BLOCK_LZ4_MASK = TERMS_DICT_BLOCK_LZ4_SIZE - 1;
static final int TERMS_DICT_COMPRESSOR_LZ4_CODE = 1;
// Writing a special code so we know this is a LZ4-compressed block.
static final int TERMS_DICT_BLOCK_LZ4_CODE =
TERMS_DICT_BLOCK_LZ4_SHIFT << 16 | TERMS_DICT_COMPRESSOR_LZ4_CODE;
static final int TERMS_DICT_REVERSE_INDEX_SHIFT = 10;
static final int TERMS_DICT_REVERSE_INDEX_SIZE = 1 << TERMS_DICT_REVERSE_INDEX_SHIFT;

View File

@ -16,6 +16,8 @@
*/
package org.apache.lucene.codecs.lucene90;
import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_SHIFT;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
@ -86,7 +88,7 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
state.segmentInfo.getId(),
state.segmentSuffix);
readFields(state.segmentInfo.name, in, state.fieldInfos);
readFields(in, state.fieldInfos);
} catch (Throwable exception) {
priorE = exception;
@ -127,8 +129,7 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
}
}
private void readFields(String segmentName, IndexInput meta, FieldInfos infos)
throws IOException {
private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
for (int fieldNumber = meta.readInt(); fieldNumber != -1; fieldNumber = meta.readInt()) {
FieldInfo info = infos.fieldInfo(fieldNumber);
if (info == null) {
@ -138,19 +139,7 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
if (type == Lucene90DocValuesFormat.NUMERIC) {
numerics.put(info.name, readNumeric(meta));
} else if (type == Lucene90DocValuesFormat.BINARY) {
String value = info.getAttribute(Lucene90DocValuesFormat.MODE_KEY);
if (value == null) {
throw new IllegalStateException(
"missing value for "
+ Lucene90DocValuesFormat.MODE_KEY
+ " for field: "
+ info.name
+ " in segment: "
+ segmentName);
}
Lucene90DocValuesFormat.Mode mode = Lucene90DocValuesFormat.Mode.valueOf(value);
final boolean compressed = mode == Lucene90DocValuesFormat.Mode.BEST_COMPRESSION;
binaries.put(info.name, readBinary(meta, compressed));
binaries.put(info.name, readBinary(meta));
} else if (type == Lucene90DocValuesFormat.SORTED) {
sorted.put(info.name, readSorted(meta));
} else if (type == Lucene90DocValuesFormat.SORTED_SET) {
@ -198,9 +187,8 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
entry.valueJumpTableOffset = meta.readLong();
}
private BinaryEntry readBinary(IndexInput meta, boolean compressed) throws IOException {
private BinaryEntry readBinary(IndexInput meta) throws IOException {
final BinaryEntry entry = new BinaryEntry();
entry.compressed = compressed;
entry.dataOffset = meta.readLong();
entry.dataLength = meta.readLong();
entry.docsWithFieldOffset = meta.readLong();
@ -210,18 +198,11 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
entry.numDocsWithField = meta.readInt();
entry.minLength = meta.readInt();
entry.maxLength = meta.readInt();
if ((entry.compressed && entry.numDocsWithField > 0) || entry.minLength < entry.maxLength) {
if (entry.minLength < entry.maxLength) {
entry.addressesOffset = meta.readLong();
// Old count of uncompressed addresses
long numAddresses = entry.numDocsWithField + 1L;
// New count of compressed addresses - the number of compresseed blocks
if (entry.compressed) {
entry.numCompressedChunks = meta.readVInt();
entry.docsPerChunkShift = meta.readVInt();
entry.maxUncompressedChunkSize = meta.readVInt();
numAddresses = entry.numCompressedChunks;
}
final int blockShift = meta.readVInt();
entry.addressesMeta = DirectMonotonicReader.loadMeta(meta, numAddresses, blockShift);
@ -275,24 +256,13 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
private static void readTermDict(IndexInput meta, TermsDictEntry entry) throws IOException {
entry.termsDictSize = meta.readVLong();
int termsDictBlockCode = meta.readInt();
if (Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_CODE == termsDictBlockCode) {
// This is a LZ4 compressed block.
entry.compressed = true;
entry.termsDictBlockShift = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_SHIFT;
} else {
entry.termsDictBlockShift = termsDictBlockCode;
}
final int blockShift = meta.readInt();
final long addressesSize =
(entry.termsDictSize + (1L << entry.termsDictBlockShift) - 1) >>> entry.termsDictBlockShift;
(entry.termsDictSize + (1L << TERMS_DICT_BLOCK_LZ4_SHIFT) - 1)
>>> TERMS_DICT_BLOCK_LZ4_SHIFT;
entry.termsAddressesMeta = DirectMonotonicReader.loadMeta(meta, addressesSize, blockShift);
entry.maxTermLength = meta.readInt();
// Read one more int for compressed term dict.
if (entry.compressed) {
entry.maxBlockLength = meta.readInt();
}
entry.maxBlockLength = meta.readInt();
entry.termsDataOffset = meta.readLong();
entry.termsDataLength = meta.readLong();
entry.termsAddressesOffset = meta.readLong();
@ -343,7 +313,6 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
}
private static class BinaryEntry {
boolean compressed;
long dataOffset;
long dataLength;
long docsWithFieldOffset;
@ -356,14 +325,10 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
long addressesOffset;
long addressesLength;
DirectMonotonicReader.Meta addressesMeta;
int numCompressedChunks;
int docsPerChunkShift;
int maxUncompressedChunkSize;
}
private static class TermsDictEntry {
long termsDictSize;
int termsDictBlockShift;
DirectMonotonicReader.Meta termsAddressesMeta;
int maxTermLength;
long termsDataOffset;
@ -377,7 +342,6 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
long termsIndexAddressesOffset;
long termsIndexAddressesLength;
boolean compressed;
int maxBlockLength;
}
@ -728,7 +692,10 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
}
}
private BinaryDocValues getUncompressedBinary(BinaryEntry entry) throws IOException {
@Override
public BinaryDocValues getBinary(FieldInfo field) throws IOException {
BinaryEntry entry = binaries.get(field.name);
if (entry.docsWithFieldOffset == -2) {
return DocValues.emptyBinary();
}
@ -815,148 +782,6 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
}
}
// Decompresses blocks of binary values to retrieve content
static class BinaryDecoder {
private final LongValues addresses;
private final IndexInput compressedData;
// Cache of last uncompressed block
private long lastBlockId = -1;
private final int[] uncompressedDocStarts;
private int uncompressedBlockLength = 0;
private final byte[] uncompressedBlock;
private final BytesRef uncompressedBytesRef;
private final int docsPerChunk;
private final int docsPerChunkShift;
public BinaryDecoder(
LongValues addresses,
IndexInput compressedData,
int biggestUncompressedBlockSize,
int docsPerChunkShift) {
super();
this.addresses = addresses;
this.compressedData = compressedData;
// pre-allocate a byte array large enough for the biggest uncompressed block needed.
this.uncompressedBlock = new byte[biggestUncompressedBlockSize];
uncompressedBytesRef = new BytesRef(uncompressedBlock);
this.docsPerChunk = 1 << docsPerChunkShift;
this.docsPerChunkShift = docsPerChunkShift;
uncompressedDocStarts = new int[docsPerChunk + 1];
}
BytesRef decode(int docNumber) throws IOException {
int blockId = docNumber >> docsPerChunkShift;
int docInBlockId = docNumber % docsPerChunk;
assert docInBlockId < docsPerChunk;
// already read and uncompressed?
if (blockId != lastBlockId) {
lastBlockId = blockId;
long blockStartOffset = addresses.get(blockId);
compressedData.seek(blockStartOffset);
uncompressedBlockLength = 0;
int onlyLength = -1;
for (int i = 0; i < docsPerChunk; i++) {
if (i == 0) {
// The first length value is special. It is shifted and has a bit to denote if
// all other values are the same length
int lengthPlusSameInd = compressedData.readVInt();
int sameIndicator = lengthPlusSameInd & 1;
int firstValLength = lengthPlusSameInd >>> 1;
if (sameIndicator == 1) {
onlyLength = firstValLength;
}
uncompressedBlockLength += firstValLength;
} else {
if (onlyLength == -1) {
// Various lengths are stored - read each from disk
uncompressedBlockLength += compressedData.readVInt();
} else {
// Only one length
uncompressedBlockLength += onlyLength;
}
}
uncompressedDocStarts[i + 1] = uncompressedBlockLength;
}
if (uncompressedBlockLength == 0) {
uncompressedBytesRef.offset = 0;
uncompressedBytesRef.length = 0;
return uncompressedBytesRef;
}
assert uncompressedBlockLength <= uncompressedBlock.length;
LZ4.decompress(compressedData, uncompressedBlockLength, uncompressedBlock, 0);
}
uncompressedBytesRef.offset = uncompressedDocStarts[docInBlockId];
uncompressedBytesRef.length =
uncompressedDocStarts[docInBlockId + 1] - uncompressedBytesRef.offset;
return uncompressedBytesRef;
}
}
@Override
public BinaryDocValues getBinary(FieldInfo field) throws IOException {
BinaryEntry entry = binaries.get(field.name);
if (entry.compressed) {
return getCompressedBinary(entry);
} else {
return getUncompressedBinary(entry);
}
}
private BinaryDocValues getCompressedBinary(BinaryEntry entry) throws IOException {
if (entry.docsWithFieldOffset == -2) {
return DocValues.emptyBinary();
}
if (entry.docsWithFieldOffset == -1) {
// dense
final RandomAccessInput addressesData =
this.data.randomAccessSlice(entry.addressesOffset, entry.addressesLength);
final LongValues addresses =
DirectMonotonicReader.getInstance(entry.addressesMeta, addressesData);
return new DenseBinaryDocValues(maxDoc) {
BinaryDecoder decoder =
new BinaryDecoder(
addresses, data.clone(), entry.maxUncompressedChunkSize, entry.docsPerChunkShift);
@Override
public BytesRef binaryValue() throws IOException {
return decoder.decode(doc);
}
};
} else {
// sparse
final IndexedDISI disi =
new IndexedDISI(
data,
entry.docsWithFieldOffset,
entry.docsWithFieldLength,
entry.jumpTableEntryCount,
entry.denseRankPower,
entry.numDocsWithField);
final RandomAccessInput addressesData =
this.data.randomAccessSlice(entry.addressesOffset, entry.addressesLength);
final LongValues addresses =
DirectMonotonicReader.getInstance(entry.addressesMeta, addressesData);
return new SparseBinaryDocValues(disi) {
BinaryDecoder decoder =
new BinaryDecoder(
addresses, data.clone(), entry.maxUncompressedChunkSize, entry.docsPerChunkShift);
@Override
public BytesRef binaryValue() throws IOException {
return decoder.decode(disi.index());
}
};
}
}
@Override
public SortedDocValues getSorted(FieldInfo field) throws IOException {
SortedEntry entry = sorted.get(field.name);
@ -1174,7 +999,7 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
data.randomAccessSlice(entry.termsAddressesOffset, entry.termsAddressesLength);
blockAddresses = DirectMonotonicReader.getInstance(entry.termsAddressesMeta, addressesSlice);
bytes = data.slice("terms", entry.termsDataOffset, entry.termsDataLength);
blockMask = (1L << entry.termsDictBlockShift) - 1;
blockMask = (1L << TERMS_DICT_BLOCK_LZ4_SHIFT) - 1;
RandomAccessInput indexAddressesSlice =
data.randomAccessSlice(entry.termsIndexAddressesOffset, entry.termsIndexAddressesLength);
indexAddresses =
@ -1182,11 +1007,9 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
indexBytes = data.slice("terms-index", entry.termsIndexOffset, entry.termsIndexLength);
term = new BytesRef(entry.maxTermLength);
if (entry.compressed) {
// add 7 padding bytes can help decompression run faster.
int bufferSize = entry.maxBlockLength + LZ4_DECOMPRESSOR_PADDING;
blockBuffer = new BytesRef(new byte[bufferSize], 0, bufferSize);
}
// add 7 padding bytes can help decompression run faster.
int bufferSize = entry.maxBlockLength + LZ4_DECOMPRESSOR_PADDING;
blockBuffer = new BytesRef(new byte[bufferSize], 0, bufferSize);
}
@Override
@ -1196,14 +1019,9 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
}
if ((ord & blockMask) == 0L) {
if (this.entry.compressed) {
decompressBlock();
} else {
term.length = bytes.readVInt();
bytes.readBytes(term.bytes, 0, term.length);
}
decompressBlock();
} else {
DataInput input = this.entry.compressed ? blockInput : bytes;
DataInput input = blockInput;
final int token = Byte.toUnsignedInt(input.readByte());
int prefixLength = token & 0x0F;
int suffixLength = 1 + (token >>> 4);
@ -1224,10 +1042,10 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
if (ord < 0 || ord >= entry.termsDictSize) {
throw new IndexOutOfBoundsException();
}
final long blockIndex = ord >>> entry.termsDictBlockShift;
final long blockIndex = ord >>> TERMS_DICT_BLOCK_LZ4_SHIFT;
final long blockAddress = blockAddresses.get(blockIndex);
bytes.seek(blockAddress);
this.ord = (blockIndex << entry.termsDictBlockShift) - 1;
this.ord = (blockIndex << TERMS_DICT_BLOCK_LZ4_SHIFT) - 1;
do {
next();
} while (this.ord < ord);
@ -1264,7 +1082,7 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
}
private BytesRef getFirstTermFromBlock(long block) throws IOException {
assert block >= 0 && block <= (entry.termsDictSize - 1) >>> entry.termsDictBlockShift;
assert block >= 0 && block <= (entry.termsDictSize - 1) >>> TERMS_DICT_BLOCK_LZ4_SHIFT;
final long blockAddress = blockAddresses.get(block);
bytes.seek(blockAddress);
term.length = bytes.readVInt();
@ -1281,8 +1099,8 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
long ordLo = index << entry.termsDictIndexShift;
long ordHi = Math.min(entry.termsDictSize, ordLo + (1L << entry.termsDictIndexShift)) - 1L;
long blockLo = ordLo >>> entry.termsDictBlockShift;
long blockHi = ordHi >>> entry.termsDictBlockShift;
long blockLo = ordLo >>> TERMS_DICT_BLOCK_LZ4_SHIFT;
long blockHi = ordHi >>> TERMS_DICT_BLOCK_LZ4_SHIFT;
while (blockLo <= blockHi) {
final long blockMid = (blockLo + blockHi) >>> 1;
@ -1296,7 +1114,7 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
}
assert blockHi < 0 || getFirstTermFromBlock(blockHi).compareTo(text) <= 0;
assert blockHi == ((entry.termsDictSize - 1) >>> entry.termsDictBlockShift)
assert blockHi == ((entry.termsDictSize - 1) >>> TERMS_DICT_BLOCK_LZ4_SHIFT)
|| getFirstTermFromBlock(blockHi + 1).compareTo(text) > 0;
return blockHi;
@ -1311,14 +1129,9 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
return SeekStatus.NOT_FOUND;
}
final long blockAddress = blockAddresses.get(block);
this.ord = block << entry.termsDictBlockShift;
this.ord = block << TERMS_DICT_BLOCK_LZ4_SHIFT;
bytes.seek(blockAddress);
if (this.entry.compressed) {
decompressBlock();
} else {
term.length = bytes.readVInt();
bytes.readBytes(term.bytes, 0, term.length);
}
decompressBlock();
while (true) {
int cmp = term.compareTo(text);

View File

@ -1,33 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene90;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.util.TestUtil;
/** Tests Lucene90DocValuesFormat */
public class TestBestCompressionLucene90DocValuesFormat
extends BaseLucene90DocValuesFormatTestCase {
private final Codec codec =
TestUtil.alwaysDocValuesFormat(
new Lucene90DocValuesFormat(Lucene90DocValuesFormat.Mode.BEST_COMPRESSION));
@Override
protected Codec getCodec() {
return codec;
}
}

View File

@ -1,32 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene90;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.util.TestUtil;
/** Tests Lucene90DocValuesFormat */
public class TestBestSpeedLucene90DocValuesFormat extends BaseLucene90DocValuesFormatTestCase {
private final Codec codec =
TestUtil.alwaysDocValuesFormat(
new Lucene90DocValuesFormat(Lucene90DocValuesFormat.Mode.BEST_SPEED));
@Override
protected Codec getCodec() {
return codec;
}
}

View File

@ -1,313 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene90;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.standard.StandardAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.SortedDocValuesField;
import org.apache.lucene.document.SortedSetDocValuesField;
import org.apache.lucene.document.StringField;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
public class TestDocValuesCompression extends LuceneTestCase {
private final Codec bestSpeed = new Lucene90Codec(Lucene90Codec.Mode.BEST_SPEED);
private final Codec bestCompression = new Lucene90Codec(Lucene90Codec.Mode.BEST_COMPRESSION);
public void testTermsDictCompressionForLowCardinalityFields() throws IOException {
final int CARDINALITY = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_COMPRESSION_THRESHOLD - 1;
Set<String> valuesSet = new HashSet<>();
for (int i = 0; i < CARDINALITY; ++i) {
final int length = TestUtil.nextInt(random(), 10, 30);
String value = TestUtil.randomSimpleString(random(), length);
valuesSet.add(value);
}
List<String> values = new ArrayList<>(valuesSet);
long sizeForBestSpeed = writeAndGetDocValueFileSize(bestSpeed, values);
long sizeForBestCompression = writeAndGetDocValueFileSize(bestCompression, values);
// Ensure terms dict data was not compressed for low-cardinality fields.
assertEquals(sizeForBestSpeed, sizeForBestCompression);
}
public void testTermsDictCompressionForHighCardinalityFields() throws IOException {
final int CARDINALITY = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_COMPRESSION_THRESHOLD << 1;
Set<String> valuesSet = new HashSet<>();
for (int i = 0; i < CARDINALITY; ++i) {
final int length = TestUtil.nextInt(random(), 10, 30);
String value = TestUtil.randomSimpleString(random(), length);
// Add common suffix for better compression ratio.
valuesSet.add(value + "_CommonPartBetterForCompression");
}
List<String> values = new ArrayList<>(valuesSet);
long sizeForBestSpeed = writeAndGetDocValueFileSize(bestSpeed, values);
long sizeForBestCompression = writeAndGetDocValueFileSize(bestCompression, values);
// Compression happened.
assertTrue(sizeForBestCompression < sizeForBestSpeed);
}
public void testReseekAfterSkipDecompression() throws IOException {
final int CARDINALITY = (Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_SIZE << 1) + 11;
Set<String> valueSet = new HashSet<>(CARDINALITY);
for (int i = 0; i < CARDINALITY; i++) {
valueSet.add(TestUtil.randomSimpleString(random(), 64));
}
List<String> values = new ArrayList<>(valueSet);
Collections.sort(values);
// Create one non-existent value just between block-1 and block-2.
String nonexistentValue =
values.get(Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_SIZE - 1)
+ TestUtil.randomSimpleString(random(), 64, 128);
int docValues = values.size();
try (Directory directory = newDirectory()) {
Analyzer analyzer = new StandardAnalyzer();
IndexWriterConfig config = new IndexWriterConfig(analyzer);
config.setCodec(bestCompression);
config.setUseCompoundFile(false);
IndexWriter writer = new IndexWriter(directory, config);
for (int i = 0; i < 280; i++) {
Document doc = new Document();
doc.add(new StringField("id", "Doc" + i, Field.Store.NO));
doc.add(new SortedDocValuesField("sdv", new BytesRef(values.get(i % docValues))));
writer.addDocument(doc);
}
writer.commit();
writer.forceMerge(1);
DirectoryReader dReader = DirectoryReader.open(writer);
writer.close();
LeafReader reader = getOnlyLeafReader(dReader);
// Check values count.
SortedDocValues ssdvMulti = reader.getSortedDocValues("sdv");
assertEquals(docValues, ssdvMulti.getValueCount());
// Seek to first block.
int ord1 = ssdvMulti.lookupTerm(new BytesRef(values.get(0)));
assertTrue(ord1 >= 0);
int ord2 = ssdvMulti.lookupTerm(new BytesRef(values.get(1)));
assertTrue(ord2 >= ord1);
// Ensure re-seek logic is correct after skip-decompression.
int nonexistentOrd2 = ssdvMulti.lookupTerm(new BytesRef(nonexistentValue));
assertTrue(nonexistentOrd2 < 0);
dReader.close();
}
}
public void testLargeTermsCompression() throws IOException {
final int CARDINALITY = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_COMPRESSION_THRESHOLD << 1;
Set<String> valuesSet = new HashSet<>();
for (int i = 0; i < CARDINALITY; ++i) {
final int length = TestUtil.nextInt(random(), 512, 1024);
valuesSet.add(TestUtil.randomSimpleString(random(), length));
}
int valuesCount = valuesSet.size();
List<String> values = new ArrayList<>(valuesSet);
try (Directory directory = newDirectory()) {
Analyzer analyzer = new StandardAnalyzer();
IndexWriterConfig config = new IndexWriterConfig(analyzer);
config.setCodec(bestCompression);
config.setUseCompoundFile(false);
IndexWriter writer = new IndexWriter(directory, config);
for (int i = 0; i < 256; i++) {
Document doc = new Document();
doc.add(new StringField("id", "Doc" + i, Field.Store.NO));
doc.add(new SortedDocValuesField("sdv", new BytesRef(values.get(i % valuesCount))));
writer.addDocument(doc);
}
writer.commit();
writer.forceMerge(1);
DirectoryReader ireader = DirectoryReader.open(writer);
writer.close();
LeafReader reader = getOnlyLeafReader(ireader);
// Check values count.
SortedDocValues ssdvMulti = reader.getSortedDocValues("sdv");
assertEquals(valuesCount, ssdvMulti.getValueCount());
ireader.close();
}
}
// Ensure the old segment can be merged together with the new compressed segment.
public void testMergeWithUncompressedSegment() throws IOException {
final int CARDINALITY = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_COMPRESSION_THRESHOLD << 1;
Set<String> valuesSet = new HashSet<>();
for (int i = 0; i < CARDINALITY; ++i) {
final int length = TestUtil.nextInt(random(), 10, 30);
// Add common suffix for better compression ratio.
valuesSet.add(TestUtil.randomSimpleString(random(), length));
}
List<String> values = new ArrayList<>(valuesSet);
int valuesCount = values.size();
try (Directory directory = newDirectory()) {
// 1. Write 256 documents without terms dict compression.
Analyzer analyzer = new StandardAnalyzer();
IndexWriterConfig config = new IndexWriterConfig(analyzer);
config.setCodec(bestSpeed);
config.setUseCompoundFile(false);
IndexWriter writer = new IndexWriter(directory, config);
for (int i = 0; i < 256; i++) {
Document doc = new Document();
doc.add(new StringField("id", "Doc" + i, Field.Store.NO));
doc.add(new SortedSetDocValuesField("ssdv", new BytesRef(values.get(i % valuesCount))));
doc.add(
new SortedSetDocValuesField("ssdv", new BytesRef(values.get((i + 1) % valuesCount))));
doc.add(
new SortedSetDocValuesField("ssdv", new BytesRef(values.get((i + 2) % valuesCount))));
doc.add(new SortedDocValuesField("sdv", new BytesRef(values.get(i % valuesCount))));
writer.addDocument(doc);
}
writer.commit();
DirectoryReader ireader = DirectoryReader.open(writer);
assertEquals(256, ireader.numDocs());
LeafReader reader = getOnlyLeafReader(ireader);
SortedSetDocValues ssdv = reader.getSortedSetDocValues("ssdv");
assertEquals(valuesCount, ssdv.getValueCount());
SortedDocValues sdv = reader.getSortedDocValues("sdv");
assertEquals(valuesCount, sdv.getValueCount());
ireader.close();
writer.close();
// 2. Add another 100 documents, and enabling terms dict compression.
config = new IndexWriterConfig(analyzer);
config.setCodec(bestCompression);
config.setUseCompoundFile(false);
writer = new IndexWriter(directory, config);
// Add 2 new values.
valuesSet.add(TestUtil.randomSimpleString(random(), 10));
valuesSet.add(TestUtil.randomSimpleString(random(), 10));
values = new ArrayList<>(valuesSet);
valuesCount = valuesSet.size();
for (int i = 256; i < 356; i++) {
Document doc = new Document();
doc.add(new StringField("id", "Doc" + i, Field.Store.NO));
doc.add(new SortedSetDocValuesField("ssdv", new BytesRef(values.get(i % valuesCount))));
doc.add(new SortedDocValuesField("sdv", new BytesRef(values.get(i % valuesCount))));
writer.addDocument(doc);
}
writer.commit();
writer.forceMerge(1);
ireader = DirectoryReader.open(writer);
assertEquals(356, ireader.numDocs());
reader = getOnlyLeafReader(ireader);
ssdv = reader.getSortedSetDocValues("ssdv");
assertEquals(valuesCount, ssdv.getValueCount());
ireader.close();
writer.close();
}
}
private static long writeAndGetDocValueFileSize(Codec codec, List<String> values)
throws IOException {
int valuesCount = values.size();
long dvdFileSize = -1;
try (Directory directory = newDirectory()) {
Analyzer analyzer = new StandardAnalyzer();
IndexWriterConfig config = new IndexWriterConfig(analyzer);
config.setCodec(codec);
config.setUseCompoundFile(false);
IndexWriter writer = new IndexWriter(directory, config);
for (int i = 0; i < 256; i++) {
Document doc = new Document();
doc.add(new StringField("id", "Doc" + i, Field.Store.NO));
// Multi value sorted-set field.
doc.add(
new SortedSetDocValuesField("ssdv_multi_", new BytesRef(values.get(i % valuesCount))));
doc.add(
new SortedSetDocValuesField(
"ssdv_multi_", new BytesRef(values.get((i + 1) % valuesCount))));
doc.add(
new SortedSetDocValuesField(
"ssdv_multi_", new BytesRef(values.get((i + 2) % valuesCount))));
// Single value sorted-set field.
doc.add(
new SortedSetDocValuesField("ssdv_single_", new BytesRef(values.get(i % valuesCount))));
// Sorted field.
doc.add(new SortedDocValuesField("sdv", new BytesRef(values.get(i % valuesCount))));
writer.addDocument(doc);
}
writer.commit();
writer.forceMerge(1);
DirectoryReader ireader = DirectoryReader.open(writer);
writer.close();
LeafReader reader = getOnlyLeafReader(ireader);
// Check values count.
SortedSetDocValues ssdvMulti = reader.getSortedSetDocValues("ssdv_multi_");
assertEquals(valuesCount, ssdvMulti.getValueCount());
for (int i = 0; i < valuesCount; i++) {
BytesRef term = ssdvMulti.lookupOrd(i);
assertTrue(term.bytes.length > 0);
}
for (int i = 0; i < valuesCount; i++) {
for (int j = 0; j < 3; j++) {
assertTrue(ssdvMulti.lookupTerm(new BytesRef(values.get((i + j) % valuesCount))) >= 0);
}
}
SortedSetDocValues ssdvSingle = reader.getSortedSetDocValues("ssdv_single_");
assertEquals(valuesCount, ssdvSingle.getValueCount());
for (int i = 0; i < valuesCount; i++) {
assertTrue(ssdvSingle.lookupTerm(new BytesRef(values.get(i % valuesCount))) >= 0);
}
SortedDocValues sdv = reader.getSortedDocValues("sdv");
assertEquals(valuesCount, sdv.getValueCount());
for (int i = 0; i < valuesCount; i++) {
assertTrue(sdv.lookupTerm(new BytesRef(values.get(i % valuesCount))) >= 0);
}
dvdFileSize = docValueFileSize(directory);
assertTrue(dvdFileSize > 0);
ireader.close();
}
return dvdFileSize;
}
static long docValueFileSize(Directory d) throws IOException {
for (String file : d.listAll()) {
if (file.endsWith(Lucene90DocValuesFormat.DATA_EXTENSION)) {
return d.fileLength(file);
}
}
return -1;
}
}

View File

@ -26,7 +26,10 @@ import java.util.Set;
import java.util.TreeSet;
import java.util.function.LongSupplier;
import java.util.function.Supplier;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.analysis.standard.StandardAnalyzer;
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;
@ -68,8 +71,13 @@ import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.TestUtil;
/** Tests Lucene90DocValuesFormat */
public abstract class BaseLucene90DocValuesFormatTestCase
extends BaseCompressingDocValuesFormatTestCase {
public class TestLucene90DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
private final Codec codec = TestUtil.getDefaultCodec();
@Override
protected Codec getCodec() {
return codec;
}
// TODO: these big methods can easily blow up some of the other ram-hungry codecs...
// for now just keep them here, as we want to test this for this format.
@ -761,4 +769,87 @@ public abstract class BaseLucene90DocValuesFormatTestCase
}
ir.close();
}
public void testReseekAfterSkipDecompression() throws IOException {
final int CARDINALITY = (Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_SIZE << 1) + 11;
Set<String> valueSet = new HashSet<>(CARDINALITY);
for (int i = 0; i < CARDINALITY; i++) {
valueSet.add(TestUtil.randomSimpleString(random(), 64));
}
List<String> values = new ArrayList<>(valueSet);
Collections.sort(values);
// Create one non-existent value just between block-1 and block-2.
String nonexistentValue =
values.get(Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_SIZE - 1)
+ TestUtil.randomSimpleString(random(), 64, 128);
int docValues = values.size();
try (Directory directory = newDirectory()) {
Analyzer analyzer = new StandardAnalyzer();
IndexWriterConfig config = new IndexWriterConfig(analyzer);
config.setCodec(getCodec());
config.setUseCompoundFile(false);
IndexWriter writer = new IndexWriter(directory, config);
for (int i = 0; i < 280; i++) {
Document doc = new Document();
doc.add(new StringField("id", "Doc" + i, Field.Store.NO));
doc.add(new SortedDocValuesField("sdv", new BytesRef(values.get(i % docValues))));
writer.addDocument(doc);
}
writer.commit();
writer.forceMerge(1);
DirectoryReader dReader = DirectoryReader.open(writer);
writer.close();
LeafReader reader = getOnlyLeafReader(dReader);
// Check values count.
SortedDocValues ssdvMulti = reader.getSortedDocValues("sdv");
assertEquals(docValues, ssdvMulti.getValueCount());
// Seek to first block.
int ord1 = ssdvMulti.lookupTerm(new BytesRef(values.get(0)));
assertTrue(ord1 >= 0);
int ord2 = ssdvMulti.lookupTerm(new BytesRef(values.get(1)));
assertTrue(ord2 >= ord1);
// Ensure re-seek logic is correct after skip-decompression.
int nonexistentOrd2 = ssdvMulti.lookupTerm(new BytesRef(nonexistentValue));
assertTrue(nonexistentOrd2 < 0);
dReader.close();
}
}
public void testLargeTermsCompression() throws IOException {
final int CARDINALITY = 64;
Set<String> valuesSet = new HashSet<>();
for (int i = 0; i < CARDINALITY; ++i) {
final int length = TestUtil.nextInt(random(), 512, 1024);
valuesSet.add(TestUtil.randomSimpleString(random(), length));
}
int valuesCount = valuesSet.size();
List<String> values = new ArrayList<>(valuesSet);
try (Directory directory = newDirectory()) {
Analyzer analyzer = new StandardAnalyzer();
IndexWriterConfig config = new IndexWriterConfig(analyzer);
config.setCodec(getCodec());
config.setUseCompoundFile(false);
IndexWriter writer = new IndexWriter(directory, config);
for (int i = 0; i < 256; i++) {
Document doc = new Document();
doc.add(new StringField("id", "Doc" + i, Field.Store.NO));
doc.add(new SortedDocValuesField("sdv", new BytesRef(values.get(i % valuesCount))));
writer.addDocument(doc);
}
writer.commit();
writer.forceMerge(1);
DirectoryReader ireader = DirectoryReader.open(writer);
writer.close();
LeafReader reader = getOnlyLeafReader(ireader);
// Check values count.
SortedDocValues ssdvMulti = reader.getSortedDocValues("sdv");
assertEquals(valuesCount, ssdvMulti.getValueCount());
ireader.close();
}
}
}

View File

@ -228,7 +228,7 @@ public class RandomCodec extends AssertingCodec {
addDocValues(
avoidCodecs,
TestUtil.getDefaultDocValuesFormat(),
new Lucene90DocValuesFormat(Lucene90DocValuesFormat.Mode.BEST_COMPRESSION),
new Lucene90DocValuesFormat(),
new AssertingDocValuesFormat());
Collections.shuffle(formats, random);