mirror of
https://github.com/apache/lucene.git
synced 2025-02-08 19:15:06 +00:00
LUCENE-9843: Remove compression option on default codec's docvalues
This commit is contained in:
parent
e2788336d4
commit
a7a02519f0
@ -114,6 +114,8 @@ API Changes
|
|||||||
* LUCENE-9948: No longer require the user to specify whether-or-not a field is multi-valued in
|
* 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)
|
LongValueFacetCounts (detect automatically based on what is indexed). (Greg Miller)
|
||||||
|
|
||||||
|
* LUCENE-9843: Remove compression option on default codec's docvalues. (Jack Conradson)
|
||||||
|
|
||||||
Improvements
|
Improvements
|
||||||
|
|
||||||
* LUCENE-9687: Hunspell support improvements: add API for spell-checking and suggestions, support compound words,
|
* LUCENE-9687: Hunspell support improvements: add API for spell-checking and suggestions, support compound words,
|
||||||
|
@ -47,18 +47,14 @@ public class Lucene90Codec extends Codec {
|
|||||||
/** Configuration option for the codec. */
|
/** Configuration option for the codec. */
|
||||||
public enum Mode {
|
public enum Mode {
|
||||||
/** Trade compression ratio for retrieval speed. */
|
/** 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. */
|
/** Trade retrieval speed for compression ratio. */
|
||||||
BEST_COMPRESSION(
|
BEST_COMPRESSION(Lucene90StoredFieldsFormat.Mode.BEST_COMPRESSION);
|
||||||
Lucene90StoredFieldsFormat.Mode.BEST_COMPRESSION,
|
|
||||||
Lucene90DocValuesFormat.Mode.BEST_COMPRESSION);
|
|
||||||
|
|
||||||
private final Lucene90StoredFieldsFormat.Mode storedMode;
|
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.storedMode = Objects.requireNonNull(storedMode);
|
||||||
this.dvMode = Objects.requireNonNull(dvMode);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -110,7 +106,7 @@ public class Lucene90Codec extends Codec {
|
|||||||
this.storedFieldsFormat =
|
this.storedFieldsFormat =
|
||||||
new Lucene90StoredFieldsFormat(Objects.requireNonNull(mode).storedMode);
|
new Lucene90StoredFieldsFormat(Objects.requireNonNull(mode).storedMode);
|
||||||
this.defaultFormat = new Lucene90PostingsFormat();
|
this.defaultFormat = new Lucene90PostingsFormat();
|
||||||
this.defaultDVFormat = new Lucene90DocValuesFormat(mode.dvMode);
|
this.defaultDVFormat = new Lucene90DocValuesFormat();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -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_SHIFT;
|
||||||
import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.NUMERIC_BLOCK_SIZE;
|
import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.NUMERIC_BLOCK_SIZE;
|
||||||
|
|
||||||
import java.io.Closeable;
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.HashMap;
|
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.DocValuesConsumer;
|
||||||
import org.apache.lucene.codecs.DocValuesProducer;
|
import org.apache.lucene.codecs.DocValuesProducer;
|
||||||
import org.apache.lucene.index.BinaryDocValues;
|
import org.apache.lucene.index.BinaryDocValues;
|
||||||
import org.apache.lucene.index.CorruptIndexException;
|
|
||||||
import org.apache.lucene.index.DocValues;
|
import org.apache.lucene.index.DocValues;
|
||||||
import org.apache.lucene.index.EmptyDocValuesProducer;
|
import org.apache.lucene.index.EmptyDocValuesProducer;
|
||||||
import org.apache.lucene.index.FieldInfo;
|
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.ByteArrayDataOutput;
|
||||||
import org.apache.lucene.store.ByteBuffersDataOutput;
|
import org.apache.lucene.store.ByteBuffersDataOutput;
|
||||||
import org.apache.lucene.store.ByteBuffersIndexOutput;
|
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.store.IndexOutput;
|
||||||
import org.apache.lucene.util.ArrayUtil;
|
import org.apache.lucene.util.ArrayUtil;
|
||||||
import org.apache.lucene.util.BytesRef;
|
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.MathUtil;
|
||||||
import org.apache.lucene.util.StringHelper;
|
import org.apache.lucene.util.StringHelper;
|
||||||
import org.apache.lucene.util.compress.LZ4;
|
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.DirectMonotonicWriter;
|
||||||
import org.apache.lucene.util.packed.DirectWriter;
|
import org.apache.lucene.util.packed.DirectWriter;
|
||||||
|
|
||||||
/** writer for {@link Lucene90DocValuesFormat} */
|
/** writer for {@link Lucene90DocValuesFormat} */
|
||||||
final class Lucene90DocValuesConsumer extends DocValuesConsumer {
|
final class Lucene90DocValuesConsumer extends DocValuesConsumer {
|
||||||
|
|
||||||
final Lucene90DocValuesFormat.Mode mode;
|
|
||||||
IndexOutput data, meta;
|
IndexOutput data, meta;
|
||||||
final int maxDoc;
|
final int maxDoc;
|
||||||
private final SegmentWriteState state;
|
|
||||||
private byte[] termsDictBuffer;
|
private byte[] termsDictBuffer;
|
||||||
|
|
||||||
/** expert: Creates a new writer */
|
/** expert: Creates a new writer */
|
||||||
@ -76,16 +68,11 @@ final class Lucene90DocValuesConsumer extends DocValuesConsumer {
|
|||||||
String dataCodec,
|
String dataCodec,
|
||||||
String dataExtension,
|
String dataExtension,
|
||||||
String metaCodec,
|
String metaCodec,
|
||||||
String metaExtension,
|
String metaExtension)
|
||||||
Lucene90DocValuesFormat.Mode mode)
|
|
||||||
throws IOException {
|
throws IOException {
|
||||||
this.mode = mode;
|
this.termsDictBuffer = new byte[1 << 14];
|
||||||
if (Lucene90DocValuesFormat.Mode.BEST_COMPRESSION == this.mode) {
|
|
||||||
this.termsDictBuffer = new byte[1 << 14];
|
|
||||||
}
|
|
||||||
boolean success = false;
|
boolean success = false;
|
||||||
try {
|
try {
|
||||||
this.state = state;
|
|
||||||
String dataName =
|
String dataName =
|
||||||
IndexFileNames.segmentFileName(
|
IndexFileNames.segmentFileName(
|
||||||
state.segmentInfo.name, state.segmentSuffix, dataExtension);
|
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
|
@Override
|
||||||
public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
|
public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
|
||||||
field.putAttribute(Lucene90DocValuesFormat.MODE_KEY, mode.name());
|
|
||||||
meta.writeInt(field.number);
|
meta.writeInt(field.number);
|
||||||
meta.writeByte(Lucene90DocValuesFormat.BINARY);
|
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);
|
BinaryDocValues values = valuesProducer.getBinary(field);
|
||||||
long start = data.getFilePointer();
|
long start = data.getFilePointer();
|
||||||
meta.writeLong(start); // dataOffset
|
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
|
@Override
|
||||||
public void addSortedField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
|
public void addSortedField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
|
||||||
meta.writeInt(field.number);
|
meta.writeInt(field.number);
|
||||||
@ -742,21 +520,10 @@ final class Lucene90DocValuesConsumer extends DocValuesConsumer {
|
|||||||
private void addTermsDict(SortedSetDocValues values) throws IOException {
|
private void addTermsDict(SortedSetDocValues values) throws IOException {
|
||||||
final long size = values.getValueCount();
|
final long size = values.getValueCount();
|
||||||
meta.writeVLong(size);
|
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);
|
meta.writeInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
|
||||||
ByteBuffersDataOutput addressBuffer = new ByteBuffersDataOutput();
|
ByteBuffersDataOutput addressBuffer = new ByteBuffersDataOutput();
|
||||||
ByteBuffersIndexOutput addressOutput =
|
ByteBuffersIndexOutput addressOutput =
|
||||||
@ -772,16 +539,12 @@ final class Lucene90DocValuesConsumer extends DocValuesConsumer {
|
|||||||
int maxLength = 0, maxBlockLength = 0;
|
int maxLength = 0, maxBlockLength = 0;
|
||||||
TermsEnum iterator = values.termsEnum();
|
TermsEnum iterator = values.termsEnum();
|
||||||
|
|
||||||
LZ4.FastCompressionHashTable ht = null;
|
LZ4.FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();
|
||||||
ByteArrayDataOutput bufferedOutput = null;
|
ByteArrayDataOutput bufferedOutput = new ByteArrayDataOutput(termsDictBuffer);
|
||||||
if (compress) {
|
|
||||||
ht = new LZ4.FastCompressionHashTable();
|
|
||||||
bufferedOutput = new ByteArrayDataOutput(termsDictBuffer);
|
|
||||||
}
|
|
||||||
|
|
||||||
for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
|
for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
|
||||||
if ((ord & blockMask) == 0) {
|
if ((ord & blockMask) == 0) {
|
||||||
if (compress && bufferedOutput.getPosition() > 0) {
|
if (bufferedOutput.getPosition() > 0) {
|
||||||
maxBlockLength =
|
maxBlockLength =
|
||||||
Math.max(maxBlockLength, compressAndGetTermsDictBlockLength(bufferedOutput, ht));
|
Math.max(maxBlockLength, compressAndGetTermsDictBlockLength(bufferedOutput, ht));
|
||||||
bufferedOutput.reset(termsDictBuffer);
|
bufferedOutput.reset(termsDictBuffer);
|
||||||
@ -794,40 +557,32 @@ final class Lucene90DocValuesConsumer extends DocValuesConsumer {
|
|||||||
final int prefixLength = StringHelper.bytesDifference(previous.get(), term);
|
final int prefixLength = StringHelper.bytesDifference(previous.get(), term);
|
||||||
final int suffixLength = term.length - prefixLength;
|
final int suffixLength = term.length - prefixLength;
|
||||||
assert suffixLength > 0; // terms are unique
|
assert suffixLength > 0; // terms are unique
|
||||||
DataOutput blockOutput;
|
// Will write (suffixLength + 1 byte + 2 vint) bytes. Grow the buffer in need.
|
||||||
if (compress) {
|
bufferedOutput = maybeGrowBuffer(bufferedOutput, suffixLength + 11);
|
||||||
// Will write (suffixLength + 1 byte + 2 vint) bytes. Grow the buffer in need.
|
bufferedOutput.writeByte(
|
||||||
bufferedOutput = maybeGrowBuffer(bufferedOutput, suffixLength + 11);
|
|
||||||
blockOutput = bufferedOutput;
|
|
||||||
} else {
|
|
||||||
blockOutput = data;
|
|
||||||
}
|
|
||||||
blockOutput.writeByte(
|
|
||||||
(byte) (Math.min(prefixLength, 15) | (Math.min(15, suffixLength - 1) << 4)));
|
(byte) (Math.min(prefixLength, 15) | (Math.min(15, suffixLength - 1) << 4)));
|
||||||
if (prefixLength >= 15) {
|
if (prefixLength >= 15) {
|
||||||
blockOutput.writeVInt(prefixLength - 15);
|
bufferedOutput.writeVInt(prefixLength - 15);
|
||||||
}
|
}
|
||||||
if (suffixLength >= 16) {
|
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);
|
maxLength = Math.max(maxLength, term.length);
|
||||||
previous.copyBytes(term);
|
previous.copyBytes(term);
|
||||||
++ord;
|
++ord;
|
||||||
}
|
}
|
||||||
// Compress and write out the last block
|
// Compress and write out the last block
|
||||||
if (compress && bufferedOutput.getPosition() > 0) {
|
if (bufferedOutput.getPosition() > 0) {
|
||||||
maxBlockLength =
|
maxBlockLength =
|
||||||
Math.max(maxBlockLength, compressAndGetTermsDictBlockLength(bufferedOutput, ht));
|
Math.max(maxBlockLength, compressAndGetTermsDictBlockLength(bufferedOutput, ht));
|
||||||
}
|
}
|
||||||
|
|
||||||
writer.finish();
|
writer.finish();
|
||||||
meta.writeInt(maxLength);
|
meta.writeInt(maxLength);
|
||||||
if (compress) {
|
// Write one more int for storing max block length.
|
||||||
// Write one more int for storing max block length. For compressed terms dict only.
|
meta.writeInt(maxBlockLength);
|
||||||
meta.writeInt(maxBlockLength);
|
|
||||||
}
|
|
||||||
meta.writeLong(start);
|
meta.writeLong(start);
|
||||||
meta.writeLong(data.getFilePointer() - start);
|
meta.writeLong(data.getFilePointer() - start);
|
||||||
start = data.getFilePointer();
|
start = data.getFilePointer();
|
||||||
|
@ -17,7 +17,6 @@
|
|||||||
package org.apache.lucene.codecs.lucene90;
|
package org.apache.lucene.codecs.lucene90;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Objects;
|
|
||||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||||
import org.apache.lucene.codecs.DocValuesFormat;
|
import org.apache.lucene.codecs.DocValuesFormat;
|
||||||
import org.apache.lucene.codecs.DocValuesProducer;
|
import org.apache.lucene.codecs.DocValuesProducer;
|
||||||
@ -139,34 +138,15 @@ import org.apache.lucene.util.packed.DirectWriter;
|
|||||||
*/
|
*/
|
||||||
public final class Lucene90DocValuesFormat extends DocValuesFormat {
|
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. */
|
/** Default constructor. */
|
||||||
public Lucene90DocValuesFormat() {
|
public Lucene90DocValuesFormat() {
|
||||||
this(Mode.BEST_SPEED);
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Constructor */
|
|
||||||
public Lucene90DocValuesFormat(Mode mode) {
|
|
||||||
super("Lucene90");
|
super("Lucene90");
|
||||||
this.mode = Objects.requireNonNull(mode);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
||||||
return new Lucene90DocValuesConsumer(
|
return new Lucene90DocValuesConsumer(
|
||||||
state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION, mode);
|
state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -194,21 +174,9 @@ public final class Lucene90DocValuesFormat extends DocValuesFormat {
|
|||||||
static final int NUMERIC_BLOCK_SHIFT = 14;
|
static final int NUMERIC_BLOCK_SHIFT = 14;
|
||||||
static final int NUMERIC_BLOCK_SIZE = 1 << NUMERIC_BLOCK_SHIFT;
|
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_SHIFT = 6;
|
||||||
static final int TERMS_DICT_BLOCK_LZ4_SIZE = 1 << TERMS_DICT_BLOCK_LZ4_SHIFT;
|
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_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_SHIFT = 10;
|
||||||
static final int TERMS_DICT_REVERSE_INDEX_SIZE = 1 << TERMS_DICT_REVERSE_INDEX_SHIFT;
|
static final int TERMS_DICT_REVERSE_INDEX_SIZE = 1 << TERMS_DICT_REVERSE_INDEX_SHIFT;
|
||||||
|
@ -16,6 +16,8 @@
|
|||||||
*/
|
*/
|
||||||
package org.apache.lucene.codecs.lucene90;
|
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.io.IOException;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
@ -86,7 +88,7 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
|
|||||||
state.segmentInfo.getId(),
|
state.segmentInfo.getId(),
|
||||||
state.segmentSuffix);
|
state.segmentSuffix);
|
||||||
|
|
||||||
readFields(state.segmentInfo.name, in, state.fieldInfos);
|
readFields(in, state.fieldInfos);
|
||||||
|
|
||||||
} catch (Throwable exception) {
|
} catch (Throwable exception) {
|
||||||
priorE = exception;
|
priorE = exception;
|
||||||
@ -127,8 +129,7 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void readFields(String segmentName, IndexInput meta, FieldInfos infos)
|
private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
|
||||||
throws IOException {
|
|
||||||
for (int fieldNumber = meta.readInt(); fieldNumber != -1; fieldNumber = meta.readInt()) {
|
for (int fieldNumber = meta.readInt(); fieldNumber != -1; fieldNumber = meta.readInt()) {
|
||||||
FieldInfo info = infos.fieldInfo(fieldNumber);
|
FieldInfo info = infos.fieldInfo(fieldNumber);
|
||||||
if (info == null) {
|
if (info == null) {
|
||||||
@ -138,19 +139,7 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
|
|||||||
if (type == Lucene90DocValuesFormat.NUMERIC) {
|
if (type == Lucene90DocValuesFormat.NUMERIC) {
|
||||||
numerics.put(info.name, readNumeric(meta));
|
numerics.put(info.name, readNumeric(meta));
|
||||||
} else if (type == Lucene90DocValuesFormat.BINARY) {
|
} else if (type == Lucene90DocValuesFormat.BINARY) {
|
||||||
String value = info.getAttribute(Lucene90DocValuesFormat.MODE_KEY);
|
binaries.put(info.name, readBinary(meta));
|
||||||
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));
|
|
||||||
} else if (type == Lucene90DocValuesFormat.SORTED) {
|
} else if (type == Lucene90DocValuesFormat.SORTED) {
|
||||||
sorted.put(info.name, readSorted(meta));
|
sorted.put(info.name, readSorted(meta));
|
||||||
} else if (type == Lucene90DocValuesFormat.SORTED_SET) {
|
} else if (type == Lucene90DocValuesFormat.SORTED_SET) {
|
||||||
@ -198,9 +187,8 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
|
|||||||
entry.valueJumpTableOffset = meta.readLong();
|
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();
|
final BinaryEntry entry = new BinaryEntry();
|
||||||
entry.compressed = compressed;
|
|
||||||
entry.dataOffset = meta.readLong();
|
entry.dataOffset = meta.readLong();
|
||||||
entry.dataLength = meta.readLong();
|
entry.dataLength = meta.readLong();
|
||||||
entry.docsWithFieldOffset = meta.readLong();
|
entry.docsWithFieldOffset = meta.readLong();
|
||||||
@ -210,18 +198,11 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
|
|||||||
entry.numDocsWithField = meta.readInt();
|
entry.numDocsWithField = meta.readInt();
|
||||||
entry.minLength = meta.readInt();
|
entry.minLength = meta.readInt();
|
||||||
entry.maxLength = 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();
|
entry.addressesOffset = meta.readLong();
|
||||||
|
|
||||||
// Old count of uncompressed addresses
|
// Old count of uncompressed addresses
|
||||||
long numAddresses = entry.numDocsWithField + 1L;
|
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();
|
final int blockShift = meta.readVInt();
|
||||||
entry.addressesMeta = DirectMonotonicReader.loadMeta(meta, numAddresses, blockShift);
|
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 {
|
private static void readTermDict(IndexInput meta, TermsDictEntry entry) throws IOException {
|
||||||
entry.termsDictSize = meta.readVLong();
|
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 int blockShift = meta.readInt();
|
||||||
final long addressesSize =
|
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.termsAddressesMeta = DirectMonotonicReader.loadMeta(meta, addressesSize, blockShift);
|
||||||
entry.maxTermLength = meta.readInt();
|
entry.maxTermLength = meta.readInt();
|
||||||
// Read one more int for compressed term dict.
|
entry.maxBlockLength = meta.readInt();
|
||||||
if (entry.compressed) {
|
|
||||||
entry.maxBlockLength = meta.readInt();
|
|
||||||
}
|
|
||||||
entry.termsDataOffset = meta.readLong();
|
entry.termsDataOffset = meta.readLong();
|
||||||
entry.termsDataLength = meta.readLong();
|
entry.termsDataLength = meta.readLong();
|
||||||
entry.termsAddressesOffset = meta.readLong();
|
entry.termsAddressesOffset = meta.readLong();
|
||||||
@ -343,7 +313,6 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
|
|||||||
}
|
}
|
||||||
|
|
||||||
private static class BinaryEntry {
|
private static class BinaryEntry {
|
||||||
boolean compressed;
|
|
||||||
long dataOffset;
|
long dataOffset;
|
||||||
long dataLength;
|
long dataLength;
|
||||||
long docsWithFieldOffset;
|
long docsWithFieldOffset;
|
||||||
@ -356,14 +325,10 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
|
|||||||
long addressesOffset;
|
long addressesOffset;
|
||||||
long addressesLength;
|
long addressesLength;
|
||||||
DirectMonotonicReader.Meta addressesMeta;
|
DirectMonotonicReader.Meta addressesMeta;
|
||||||
int numCompressedChunks;
|
|
||||||
int docsPerChunkShift;
|
|
||||||
int maxUncompressedChunkSize;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class TermsDictEntry {
|
private static class TermsDictEntry {
|
||||||
long termsDictSize;
|
long termsDictSize;
|
||||||
int termsDictBlockShift;
|
|
||||||
DirectMonotonicReader.Meta termsAddressesMeta;
|
DirectMonotonicReader.Meta termsAddressesMeta;
|
||||||
int maxTermLength;
|
int maxTermLength;
|
||||||
long termsDataOffset;
|
long termsDataOffset;
|
||||||
@ -377,7 +342,6 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
|
|||||||
long termsIndexAddressesOffset;
|
long termsIndexAddressesOffset;
|
||||||
long termsIndexAddressesLength;
|
long termsIndexAddressesLength;
|
||||||
|
|
||||||
boolean compressed;
|
|
||||||
int maxBlockLength;
|
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) {
|
if (entry.docsWithFieldOffset == -2) {
|
||||||
return DocValues.emptyBinary();
|
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
|
@Override
|
||||||
public SortedDocValues getSorted(FieldInfo field) throws IOException {
|
public SortedDocValues getSorted(FieldInfo field) throws IOException {
|
||||||
SortedEntry entry = sorted.get(field.name);
|
SortedEntry entry = sorted.get(field.name);
|
||||||
@ -1174,7 +999,7 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
|
|||||||
data.randomAccessSlice(entry.termsAddressesOffset, entry.termsAddressesLength);
|
data.randomAccessSlice(entry.termsAddressesOffset, entry.termsAddressesLength);
|
||||||
blockAddresses = DirectMonotonicReader.getInstance(entry.termsAddressesMeta, addressesSlice);
|
blockAddresses = DirectMonotonicReader.getInstance(entry.termsAddressesMeta, addressesSlice);
|
||||||
bytes = data.slice("terms", entry.termsDataOffset, entry.termsDataLength);
|
bytes = data.slice("terms", entry.termsDataOffset, entry.termsDataLength);
|
||||||
blockMask = (1L << entry.termsDictBlockShift) - 1;
|
blockMask = (1L << TERMS_DICT_BLOCK_LZ4_SHIFT) - 1;
|
||||||
RandomAccessInput indexAddressesSlice =
|
RandomAccessInput indexAddressesSlice =
|
||||||
data.randomAccessSlice(entry.termsIndexAddressesOffset, entry.termsIndexAddressesLength);
|
data.randomAccessSlice(entry.termsIndexAddressesOffset, entry.termsIndexAddressesLength);
|
||||||
indexAddresses =
|
indexAddresses =
|
||||||
@ -1182,11 +1007,9 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
|
|||||||
indexBytes = data.slice("terms-index", entry.termsIndexOffset, entry.termsIndexLength);
|
indexBytes = data.slice("terms-index", entry.termsIndexOffset, entry.termsIndexLength);
|
||||||
term = new BytesRef(entry.maxTermLength);
|
term = new BytesRef(entry.maxTermLength);
|
||||||
|
|
||||||
if (entry.compressed) {
|
// add 7 padding bytes can help decompression run faster.
|
||||||
// add 7 padding bytes can help decompression run faster.
|
int bufferSize = entry.maxBlockLength + LZ4_DECOMPRESSOR_PADDING;
|
||||||
int bufferSize = entry.maxBlockLength + LZ4_DECOMPRESSOR_PADDING;
|
blockBuffer = new BytesRef(new byte[bufferSize], 0, bufferSize);
|
||||||
blockBuffer = new BytesRef(new byte[bufferSize], 0, bufferSize);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -1196,14 +1019,9 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
|
|||||||
}
|
}
|
||||||
|
|
||||||
if ((ord & blockMask) == 0L) {
|
if ((ord & blockMask) == 0L) {
|
||||||
if (this.entry.compressed) {
|
decompressBlock();
|
||||||
decompressBlock();
|
|
||||||
} else {
|
|
||||||
term.length = bytes.readVInt();
|
|
||||||
bytes.readBytes(term.bytes, 0, term.length);
|
|
||||||
}
|
|
||||||
} else {
|
} else {
|
||||||
DataInput input = this.entry.compressed ? blockInput : bytes;
|
DataInput input = blockInput;
|
||||||
final int token = Byte.toUnsignedInt(input.readByte());
|
final int token = Byte.toUnsignedInt(input.readByte());
|
||||||
int prefixLength = token & 0x0F;
|
int prefixLength = token & 0x0F;
|
||||||
int suffixLength = 1 + (token >>> 4);
|
int suffixLength = 1 + (token >>> 4);
|
||||||
@ -1224,10 +1042,10 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
|
|||||||
if (ord < 0 || ord >= entry.termsDictSize) {
|
if (ord < 0 || ord >= entry.termsDictSize) {
|
||||||
throw new IndexOutOfBoundsException();
|
throw new IndexOutOfBoundsException();
|
||||||
}
|
}
|
||||||
final long blockIndex = ord >>> entry.termsDictBlockShift;
|
final long blockIndex = ord >>> TERMS_DICT_BLOCK_LZ4_SHIFT;
|
||||||
final long blockAddress = blockAddresses.get(blockIndex);
|
final long blockAddress = blockAddresses.get(blockIndex);
|
||||||
bytes.seek(blockAddress);
|
bytes.seek(blockAddress);
|
||||||
this.ord = (blockIndex << entry.termsDictBlockShift) - 1;
|
this.ord = (blockIndex << TERMS_DICT_BLOCK_LZ4_SHIFT) - 1;
|
||||||
do {
|
do {
|
||||||
next();
|
next();
|
||||||
} while (this.ord < ord);
|
} while (this.ord < ord);
|
||||||
@ -1264,7 +1082,7 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
|
|||||||
}
|
}
|
||||||
|
|
||||||
private BytesRef getFirstTermFromBlock(long block) throws IOException {
|
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);
|
final long blockAddress = blockAddresses.get(block);
|
||||||
bytes.seek(blockAddress);
|
bytes.seek(blockAddress);
|
||||||
term.length = bytes.readVInt();
|
term.length = bytes.readVInt();
|
||||||
@ -1281,8 +1099,8 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
|
|||||||
long ordLo = index << entry.termsDictIndexShift;
|
long ordLo = index << entry.termsDictIndexShift;
|
||||||
long ordHi = Math.min(entry.termsDictSize, ordLo + (1L << entry.termsDictIndexShift)) - 1L;
|
long ordHi = Math.min(entry.termsDictSize, ordLo + (1L << entry.termsDictIndexShift)) - 1L;
|
||||||
|
|
||||||
long blockLo = ordLo >>> entry.termsDictBlockShift;
|
long blockLo = ordLo >>> TERMS_DICT_BLOCK_LZ4_SHIFT;
|
||||||
long blockHi = ordHi >>> entry.termsDictBlockShift;
|
long blockHi = ordHi >>> TERMS_DICT_BLOCK_LZ4_SHIFT;
|
||||||
|
|
||||||
while (blockLo <= blockHi) {
|
while (blockLo <= blockHi) {
|
||||||
final long blockMid = (blockLo + blockHi) >>> 1;
|
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 < 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;
|
|| getFirstTermFromBlock(blockHi + 1).compareTo(text) > 0;
|
||||||
|
|
||||||
return blockHi;
|
return blockHi;
|
||||||
@ -1311,14 +1129,9 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
|
|||||||
return SeekStatus.NOT_FOUND;
|
return SeekStatus.NOT_FOUND;
|
||||||
}
|
}
|
||||||
final long blockAddress = blockAddresses.get(block);
|
final long blockAddress = blockAddresses.get(block);
|
||||||
this.ord = block << entry.termsDictBlockShift;
|
this.ord = block << TERMS_DICT_BLOCK_LZ4_SHIFT;
|
||||||
bytes.seek(blockAddress);
|
bytes.seek(blockAddress);
|
||||||
if (this.entry.compressed) {
|
decompressBlock();
|
||||||
decompressBlock();
|
|
||||||
} else {
|
|
||||||
term.length = bytes.readVInt();
|
|
||||||
bytes.readBytes(term.bytes, 0, term.length);
|
|
||||||
}
|
|
||||||
|
|
||||||
while (true) {
|
while (true) {
|
||||||
int cmp = term.compareTo(text);
|
int cmp = term.compareTo(text);
|
||||||
|
@ -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;
|
|
||||||
}
|
|
||||||
}
|
|
@ -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;
|
|
||||||
}
|
|
||||||
}
|
|
@ -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;
|
|
||||||
}
|
|
||||||
}
|
|
@ -26,7 +26,10 @@ import java.util.Set;
|
|||||||
import java.util.TreeSet;
|
import java.util.TreeSet;
|
||||||
import java.util.function.LongSupplier;
|
import java.util.function.LongSupplier;
|
||||||
import java.util.function.Supplier;
|
import java.util.function.Supplier;
|
||||||
|
import org.apache.lucene.analysis.Analyzer;
|
||||||
import org.apache.lucene.analysis.MockAnalyzer;
|
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.DocValuesFormat;
|
||||||
import org.apache.lucene.codecs.PostingsFormat;
|
import org.apache.lucene.codecs.PostingsFormat;
|
||||||
import org.apache.lucene.codecs.asserting.AssertingCodec;
|
import org.apache.lucene.codecs.asserting.AssertingCodec;
|
||||||
@ -68,8 +71,13 @@ import org.apache.lucene.util.BytesRefBuilder;
|
|||||||
import org.apache.lucene.util.TestUtil;
|
import org.apache.lucene.util.TestUtil;
|
||||||
|
|
||||||
/** Tests Lucene90DocValuesFormat */
|
/** Tests Lucene90DocValuesFormat */
|
||||||
public abstract class BaseLucene90DocValuesFormatTestCase
|
public class TestLucene90DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
|
||||||
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...
|
// 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.
|
// 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();
|
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();
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
@ -228,7 +228,7 @@ public class RandomCodec extends AssertingCodec {
|
|||||||
addDocValues(
|
addDocValues(
|
||||||
avoidCodecs,
|
avoidCodecs,
|
||||||
TestUtil.getDefaultDocValuesFormat(),
|
TestUtil.getDefaultDocValuesFormat(),
|
||||||
new Lucene90DocValuesFormat(Lucene90DocValuesFormat.Mode.BEST_COMPRESSION),
|
new Lucene90DocValuesFormat(),
|
||||||
new AssertingDocValuesFormat());
|
new AssertingDocValuesFormat());
|
||||||
|
|
||||||
Collections.shuffle(formats, random);
|
Collections.shuffle(formats, random);
|
||||||
|
Loading…
x
Reference in New Issue
Block a user