mirror of https://github.com/apache/lucene.git
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
|
||||
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,
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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.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();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
|
|
Loading…
Reference in New Issue