LUCENE-9705: Create Lucene90StoredFieldsFormat (#2444)

This commit is contained in:
Ignacio Vera 2021-03-09 08:11:59 +01:00 committed by GitHub
parent cf1025e576
commit 144ef2a0c0
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
49 changed files with 2878 additions and 422 deletions

View File

@ -18,10 +18,10 @@ package org.apache.lucene.backward_codecs.lucene50;
import java.io.IOException;
import java.util.Objects;
import org.apache.lucene.backward_codecs.lucene50.compressing.Lucene50CompressingStoredFieldsFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.StoredFieldsWriter;
import org.apache.lucene.codecs.compressing.CompressingStoredFieldsFormat;
import org.apache.lucene.codecs.compressing.CompressionMode;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.SegmentInfo;
@ -147,10 +147,10 @@ public class Lucene50StoredFieldsFormat extends StoredFieldsFormat {
StoredFieldsFormat impl(Mode mode) {
switch (mode) {
case BEST_SPEED:
return new CompressingStoredFieldsFormat(
return new Lucene50CompressingStoredFieldsFormat(
"Lucene50StoredFieldsFastData", CompressionMode.FAST, 1 << 14, 128, 10);
case BEST_COMPRESSION:
return new CompressingStoredFieldsFormat(
return new Lucene50CompressingStoredFieldsFormat(
"Lucene50StoredFieldsHighData", CompressionMode.HIGH_COMPRESSION, 61440, 512, 10);
default:
throw new AssertionError();

View File

@ -17,11 +17,10 @@
package org.apache.lucene.backward_codecs.lucene50;
import org.apache.lucene.backward_codecs.lucene50.compressing.Lucene50CompressingTermVectorsFormat;
import org.apache.lucene.backward_codecs.lucene87.Lucene87StoredFieldsFormat;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.compressing.CompressionMode;
import org.apache.lucene.codecs.compressing.FieldsIndexWriter;
import org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.packed.BlockPackedWriter;
import org.apache.lucene.util.packed.PackedInts;
@ -144,7 +143,7 @@ import org.apache.lucene.util.packed.PackedInts;
* <ul>
* <li>VectorIndex (.tvx) --&gt; &lt;Header&gt;, &lt;ChunkIndex&gt;, Footer
* <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}
* <li>ChunkIndex: See {@link FieldsIndexWriter}
* <li>ChunkIndex: See FieldsIndexWriter
* <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}
* </ul>
* </ol>

View File

@ -0,0 +1,162 @@
/*
* 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.backward_codecs.lucene50.compressing;
import java.io.IOException;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.StoredFieldsWriter;
import org.apache.lucene.codecs.compressing.CompressionMode;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.MergePolicy;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.packed.DirectMonotonicWriter;
/**
* A {@link StoredFieldsFormat} that compresses documents in chunks in order to improve the
* compression ratio.
*
* <p>For a chunk size of <var>chunkSize</var> bytes, this {@link StoredFieldsFormat} does not
* support documents larger than (<code>2<sup>31</sup> - chunkSize</code>) bytes.
*
* <p>For optimal performance, you should use a {@link MergePolicy} that returns segments that have
* the biggest byte size first.
*
* @lucene.experimental
*/
public class Lucene50CompressingStoredFieldsFormat extends StoredFieldsFormat {
/** format name */
protected final String formatName;
/** segment suffix */
protected final String segmentSuffix;
/** compression mode */
protected final CompressionMode compressionMode;
/** chunk size */
protected final int chunkSize;
/** max docs per chunk */
protected final int maxDocsPerChunk;
/** block shift */
protected final int blockShift;
/**
* Create a new {@link Lucene50CompressingStoredFieldsFormat} with an empty segment suffix.
*
* @see Lucene50CompressingStoredFieldsFormat#Lucene50CompressingStoredFieldsFormat(String,
* String, CompressionMode, int, int, int)
*/
public Lucene50CompressingStoredFieldsFormat(
String formatName,
CompressionMode compressionMode,
int chunkSize,
int maxDocsPerChunk,
int blockShift) {
this(formatName, "", compressionMode, chunkSize, maxDocsPerChunk, blockShift);
}
/**
* Create a new {@link Lucene50CompressingStoredFieldsFormat}.
*
* <p><code>formatName</code> is the name of the format. This name will be used in the file
* formats to perform {@link CodecUtil#checkIndexHeader codec header checks}.
*
* <p><code>segmentSuffix</code> is the segment suffix. This suffix is added to the result file
* name only if it's not the empty string.
*
* <p>The <code>compressionMode</code> parameter allows you to choose between compression
* algorithms that have various compression and decompression speeds so that you can pick the one
* that best fits your indexing and searching throughput. You should never instantiate two {@link
* Lucene50CompressingStoredFieldsFormat}s that have the same name but different {@link
* CompressionMode}s.
*
* <p><code>chunkSize</code> is the minimum byte size of a chunk of documents. A value of <code>1
* </code> can make sense if there is redundancy across fields. <code>maxDocsPerChunk</code> is an
* upperbound on how many docs may be stored in a single chunk. This is to bound the cpu costs for
* highly compressible data.
*
* <p>Higher values of <code>chunkSize</code> should improve the compression ratio but will
* require more memory at indexing time and might make document loading a little slower (depending
* on the size of your OS cache compared to the size of your index).
*
* @param formatName the name of the {@link StoredFieldsFormat}
* @param compressionMode the {@link CompressionMode} to use
* @param chunkSize the minimum number of bytes of a single chunk of stored documents
* @param maxDocsPerChunk the maximum number of documents in a single chunk
* @param blockShift the log in base 2 of number of chunks to store in an index block
* @see CompressionMode
*/
public Lucene50CompressingStoredFieldsFormat(
String formatName,
String segmentSuffix,
CompressionMode compressionMode,
int chunkSize,
int maxDocsPerChunk,
int blockShift) {
this.formatName = formatName;
this.segmentSuffix = segmentSuffix;
this.compressionMode = compressionMode;
if (chunkSize < 1) {
throw new IllegalArgumentException("chunkSize must be >= 1");
}
this.chunkSize = chunkSize;
if (maxDocsPerChunk < 1) {
throw new IllegalArgumentException("maxDocsPerChunk must be >= 1");
}
this.maxDocsPerChunk = maxDocsPerChunk;
if (blockShift < DirectMonotonicWriter.MIN_BLOCK_SHIFT
|| blockShift > DirectMonotonicWriter.MAX_BLOCK_SHIFT) {
throw new IllegalArgumentException(
"blockSize must be in "
+ DirectMonotonicWriter.MIN_BLOCK_SHIFT
+ "-"
+ DirectMonotonicWriter.MAX_BLOCK_SHIFT
+ ", got "
+ blockShift);
}
this.blockShift = blockShift;
}
@Override
public StoredFieldsReader fieldsReader(
Directory directory, SegmentInfo si, FieldInfos fn, IOContext context) throws IOException {
return new Lucene50CompressingStoredFieldsReader(
directory, si, segmentSuffix, fn, context, formatName, compressionMode);
}
@Override
public StoredFieldsWriter fieldsWriter(Directory directory, SegmentInfo si, IOContext context)
throws IOException {
throw new UnsupportedOperationException("Old formats can't be used for writing");
}
@Override
public String toString() {
return getClass().getSimpleName()
+ "(compressionMode="
+ compressionMode
+ ", chunkSize="
+ chunkSize
+ ", maxDocsPerChunk="
+ maxDocsPerChunk
+ ", blockShift="
+ blockShift
+ ")";
}
}

View File

@ -14,31 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.compressing;
import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.BYTE_ARR;
import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.DAY;
import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.DAY_ENCODING;
import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.FIELDS_EXTENSION;
import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.HOUR;
import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.HOUR_ENCODING;
import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.INDEX_CODEC_NAME;
import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.INDEX_EXTENSION;
import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.META_EXTENSION;
import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.META_VERSION_START;
import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.NUMERIC_DOUBLE;
import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.NUMERIC_FLOAT;
import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.NUMERIC_INT;
import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.NUMERIC_LONG;
import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.SECOND;
import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.SECOND_ENCODING;
import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.STRING;
import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.TYPE_BITS;
import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.TYPE_MASK;
import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.VERSION_CURRENT;
import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.VERSION_META;
import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.VERSION_OFFHEAP_INDEX;
import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.VERSION_START;
package org.apache.lucene.backward_codecs.lucene50.compressing;
import java.io.EOFException;
import java.io.IOException;
@ -47,6 +23,8 @@ import java.util.Collection;
import java.util.Collections;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.compressing.CompressionMode;
import org.apache.lucene.codecs.compressing.Decompressor;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo;
@ -71,11 +49,46 @@ import org.apache.lucene.util.LongsRef;
import org.apache.lucene.util.packed.PackedInts;
/**
* {@link StoredFieldsReader} impl for {@link CompressingStoredFieldsFormat}.
* {@link StoredFieldsReader} impl for {@link Lucene50CompressingStoredFieldsFormat}.
*
* @lucene.experimental
*/
public final class CompressingStoredFieldsReader extends StoredFieldsReader {
public final class Lucene50CompressingStoredFieldsReader extends StoredFieldsReader {
/** Extension of stored fields file */
public static final String FIELDS_EXTENSION = "fdt";
/** Extension of stored fields index */
public static final String INDEX_EXTENSION = "fdx";
/** Extension of stored fields meta */
public static final String META_EXTENSION = "fdm";
/** Codec name for the index. */
public static final String INDEX_CODEC_NAME = "Lucene85FieldsIndex";
static final int STRING = 0x00;
static final int BYTE_ARR = 0x01;
static final int NUMERIC_INT = 0x02;
static final int NUMERIC_FLOAT = 0x03;
static final int NUMERIC_LONG = 0x04;
static final int NUMERIC_DOUBLE = 0x05;
static final int TYPE_BITS = PackedInts.bitsRequired(NUMERIC_DOUBLE);
static final int TYPE_MASK = (int) PackedInts.maxValue(TYPE_BITS);
static final int VERSION_START = 1;
static final int VERSION_OFFHEAP_INDEX = 2;
/** Version where all metadata were moved to the meta file. */
static final int VERSION_META = 3;
static final int VERSION_CURRENT = VERSION_META;
static final int META_VERSION_START = 0;
// for compression of timestamps
static final long SECOND = 1000L;
static final long HOUR = 60 * 60 * SECOND;
static final long DAY = 24 * HOUR;
static final int SECOND_ENCODING = 0x40;
static final int HOUR_ENCODING = 0x80;
static final int DAY_ENCODING = 0xC0;
private final int version;
private final FieldInfos fieldInfos;
@ -94,7 +107,8 @@ public final class CompressingStoredFieldsReader extends StoredFieldsReader {
private boolean closed;
// used by clone
private CompressingStoredFieldsReader(CompressingStoredFieldsReader reader, boolean merging) {
private Lucene50CompressingStoredFieldsReader(
Lucene50CompressingStoredFieldsReader reader, boolean merging) {
this.version = reader.version;
this.fieldInfos = reader.fieldInfos;
this.fieldsStream = reader.fieldsStream.clone();
@ -113,7 +127,7 @@ public final class CompressingStoredFieldsReader extends StoredFieldsReader {
}
/** Sole constructor. */
public CompressingStoredFieldsReader(
public Lucene50CompressingStoredFieldsReader(
Directory d,
SegmentInfo si,
String segmentSuffix,
@ -718,13 +732,13 @@ public final class CompressingStoredFieldsReader extends StoredFieldsReader {
@Override
public StoredFieldsReader clone() {
ensureOpen();
return new CompressingStoredFieldsReader(this, false);
return new Lucene50CompressingStoredFieldsReader(this, false);
}
@Override
public StoredFieldsReader getMergeInstance() {
ensureOpen();
return new CompressingStoredFieldsReader(this, true);
return new Lucene50CompressingStoredFieldsReader(this, true);
}
int getVersion() {

View File

@ -0,0 +1,109 @@
/*
* 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.backward_codecs.lucene87;
import java.util.Arrays;
import java.util.zip.DataFormatException;
import java.util.zip.Deflater;
import java.util.zip.Inflater;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.SuppressForbidden;
/**
* This class is a workaround for JDK bug <a
* href="https://bugs.openjdk.java.net/browse/JDK-8252739">JDK-8252739</a>.
*
* @lucene.internal
*/
@FunctionalInterface
interface BugfixDeflater_JDK8252739 {
public static final boolean IS_BUGGY_JDK = detectBuggyJDK();
/**
* Creates a bugfix for {@link Deflater} instances, which works around JDK-8252739.
*
* <p>Use this whenever you intend to call {@link Deflater#setDictionary(byte[], int, int)} on a
* {@code Deflater}.
*/
@SuppressForbidden(reason = "Works around bug, so it must call forbidden method")
public static BugfixDeflater_JDK8252739 createBugfix(Deflater deflater) {
if (IS_BUGGY_JDK) {
final BytesRefBuilder dictBytesScratch = new BytesRefBuilder();
return (dictBytes, off, len) -> {
if (off > 0) {
dictBytesScratch.grow(len);
System.arraycopy(dictBytes, off, dictBytesScratch.bytes(), 0, len);
deflater.setDictionary(dictBytesScratch.bytes(), 0, len);
} else {
deflater.setDictionary(dictBytes, off, len);
}
};
} else {
return deflater::setDictionary;
}
}
/** Call this method as a workaround */
void setDictionary(byte[] dictBytes, int off, int len);
@SuppressForbidden(reason = "Detector for the bug, so it must call buggy method")
private static boolean detectBuggyJDK() {
final byte[] testData = new byte[] {1, 2, 3, 4, 5, 6, 7, 8};
final byte[] compressed = new byte[32]; // way enough space
final Deflater deflater = new Deflater(6, true);
int compressedSize;
try {
deflater.reset();
deflater.setDictionary(testData, 4, 4);
deflater.setInput(testData);
deflater.finish();
compressedSize = deflater.deflate(compressed, 0, compressed.length, Deflater.FULL_FLUSH);
} finally {
deflater.end();
}
// in nowrap mode we need extra 0-byte as padding, add explicit:
compressed[compressedSize] = 0;
compressedSize++;
final Inflater inflater = new Inflater(true);
final byte[] restored = new byte[testData.length];
try {
inflater.reset();
inflater.setDictionary(testData, 4, 4);
inflater.setInput(compressed, 0, compressedSize);
final int restoredLength = inflater.inflate(restored);
if (restoredLength != testData.length) {
return true;
}
} catch (DataFormatException e) {
return true;
} catch (RuntimeException e) {
return true;
} finally {
inflater.end();
}
if (Arrays.equals(testData, restored) == false) {
return true;
}
// all fine
return false;
}
}

View File

@ -0,0 +1,230 @@
/*
* 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.backward_codecs.lucene87;
import java.io.IOException;
import java.util.zip.DataFormatException;
import java.util.zip.Deflater;
import java.util.zip.Inflater;
import org.apache.lucene.codecs.compressing.CompressionMode;
import org.apache.lucene.codecs.compressing.Compressor;
import org.apache.lucene.codecs.compressing.Decompressor;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
/**
* A compression mode that trades speed for compression ratio. Although compression and
* decompression might be slow, this compression mode should provide a good compression ratio. This
* mode might be interesting if/when your index size is much bigger than your OS cache.
*
* @lucene.internal
*/
public final class DeflateWithPresetDictCompressionMode extends CompressionMode {
// Shoot for 10 sub blocks
private static final int NUM_SUB_BLOCKS = 10;
// And a dictionary whose size is about 6x smaller than sub blocks
private static final int DICT_SIZE_FACTOR = 6;
/** Sole constructor. */
public DeflateWithPresetDictCompressionMode() {}
@Override
public Compressor newCompressor() {
// notes:
// 3 is the highest level that doesn't have lazy match evaluation
// 6 is the default, higher than that is just a waste of cpu
return new DeflateWithPresetDictCompressor(6);
}
@Override
public Decompressor newDecompressor() {
return new DeflateWithPresetDictDecompressor();
}
@Override
public String toString() {
return "BEST_COMPRESSION";
}
private static final class DeflateWithPresetDictDecompressor extends Decompressor {
byte[] compressed;
DeflateWithPresetDictDecompressor() {
compressed = new byte[0];
}
private void doDecompress(DataInput in, Inflater decompressor, BytesRef bytes)
throws IOException {
final int compressedLength = in.readVInt();
if (compressedLength == 0) {
return;
}
// pad with extra "dummy byte": see javadocs for using Inflater(true)
// we do it for compliance, but it's unnecessary for years in zlib.
final int paddedLength = compressedLength + 1;
compressed = ArrayUtil.grow(compressed, paddedLength);
in.readBytes(compressed, 0, compressedLength);
compressed[compressedLength] = 0; // explicitly set dummy byte to 0
// extra "dummy byte"
decompressor.setInput(compressed, 0, paddedLength);
try {
bytes.length +=
decompressor.inflate(bytes.bytes, bytes.length, bytes.bytes.length - bytes.length);
} catch (DataFormatException e) {
throw new IOException(e);
}
if (decompressor.finished() == false) {
throw new CorruptIndexException(
"Invalid decoder state: needsInput="
+ decompressor.needsInput()
+ ", needsDict="
+ decompressor.needsDictionary(),
in);
}
}
@Override
public void decompress(DataInput in, int originalLength, int offset, int length, BytesRef bytes)
throws IOException {
assert offset + length <= originalLength;
if (length == 0) {
bytes.length = 0;
return;
}
final int dictLength = in.readVInt();
final int blockLength = in.readVInt();
bytes.bytes = ArrayUtil.grow(bytes.bytes, dictLength);
bytes.offset = bytes.length = 0;
final Inflater decompressor = new Inflater(true);
try {
// Read the dictionary
doDecompress(in, decompressor, bytes);
if (dictLength != bytes.length) {
throw new CorruptIndexException("Unexpected dict length", in);
}
int offsetInBlock = dictLength;
int offsetInBytesRef = offset;
// Skip unneeded blocks
while (offsetInBlock + blockLength < offset) {
final int compressedLength = in.readVInt();
in.skipBytes(compressedLength);
offsetInBlock += blockLength;
offsetInBytesRef -= blockLength;
}
// Read blocks that intersect with the interval we need
while (offsetInBlock < offset + length) {
bytes.bytes = ArrayUtil.grow(bytes.bytes, bytes.length + blockLength);
decompressor.reset();
decompressor.setDictionary(bytes.bytes, 0, dictLength);
doDecompress(in, decompressor, bytes);
offsetInBlock += blockLength;
}
bytes.offset = offsetInBytesRef;
bytes.length = length;
assert bytes.isValid();
} finally {
decompressor.end();
}
}
@Override
public Decompressor clone() {
return new DeflateWithPresetDictDecompressor();
}
}
private static class DeflateWithPresetDictCompressor extends Compressor {
final Deflater compressor;
final BugfixDeflater_JDK8252739 deflaterBugfix;
byte[] compressed;
boolean closed;
DeflateWithPresetDictCompressor(int level) {
compressor = new Deflater(level, true);
deflaterBugfix = BugfixDeflater_JDK8252739.createBugfix(compressor);
compressed = new byte[64];
}
private void doCompress(byte[] bytes, int off, int len, DataOutput out) throws IOException {
if (len == 0) {
out.writeVInt(0);
return;
}
compressor.setInput(bytes, off, len);
compressor.finish();
if (compressor.needsInput()) {
throw new IllegalStateException();
}
int totalCount = 0;
for (; ; ) {
final int count =
compressor.deflate(compressed, totalCount, compressed.length - totalCount);
totalCount += count;
assert totalCount <= compressed.length;
if (compressor.finished()) {
break;
} else {
compressed = ArrayUtil.grow(compressed);
}
}
out.writeVInt(totalCount);
out.writeBytes(compressed, totalCount);
}
@Override
public void compress(byte[] bytes, int off, int len, DataOutput out) throws IOException {
final int dictLength = len / (NUM_SUB_BLOCKS * DICT_SIZE_FACTOR);
final int blockLength = (len - dictLength + NUM_SUB_BLOCKS - 1) / NUM_SUB_BLOCKS;
out.writeVInt(dictLength);
out.writeVInt(blockLength);
final int end = off + len;
// Compress the dictionary first
compressor.reset();
doCompress(bytes, off, dictLength, out);
// And then sub blocks
for (int start = off + dictLength; start < end; start += blockLength) {
compressor.reset();
deflaterBugfix.setDictionary(bytes, off, dictLength);
doCompress(bytes, start, Math.min(blockLength, off + len - start), out);
}
}
@Override
public void close() throws IOException {
if (closed == false) {
compressor.end();
closed = true;
}
}
}
}

View File

@ -0,0 +1,198 @@
/*
* 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.backward_codecs.lucene87;
import java.io.IOException;
import org.apache.lucene.codecs.compressing.CompressionMode;
import org.apache.lucene.codecs.compressing.Compressor;
import org.apache.lucene.codecs.compressing.Decompressor;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.compress.LZ4;
/**
* A compression mode that compromises on the compression ratio to provide fast compression and
* decompression.
*
* @lucene.internal
*/
public final class LZ4WithPresetDictCompressionMode extends CompressionMode {
// Shoot for 10 sub blocks
private static final int NUM_SUB_BLOCKS = 10;
// And a dictionary whose size is about 16x smaller than sub blocks
private static final int DICT_SIZE_FACTOR = 16;
/** Sole constructor. */
public LZ4WithPresetDictCompressionMode() {}
@Override
public Compressor newCompressor() {
return new LZ4WithPresetDictCompressor();
}
@Override
public Decompressor newDecompressor() {
return new LZ4WithPresetDictDecompressor();
}
@Override
public String toString() {
return "BEST_SPEED";
}
private static final class LZ4WithPresetDictDecompressor extends Decompressor {
private int[] compressedLengths;
private byte[] buffer;
LZ4WithPresetDictDecompressor() {
compressedLengths = new int[0];
buffer = new byte[0];
}
private int readCompressedLengths(
DataInput in, int originalLength, int dictLength, int blockLength) throws IOException {
in.readVInt(); // compressed length of the dictionary, unused
int totalLength = dictLength;
int i = 0;
while (totalLength < originalLength) {
compressedLengths = ArrayUtil.grow(compressedLengths, i + 1);
compressedLengths[i++] = in.readVInt();
totalLength += blockLength;
}
return i;
}
@Override
public void decompress(DataInput in, int originalLength, int offset, int length, BytesRef bytes)
throws IOException {
assert offset + length <= originalLength;
if (length == 0) {
bytes.length = 0;
return;
}
final int dictLength = in.readVInt();
final int blockLength = in.readVInt();
final int numBlocks = readCompressedLengths(in, originalLength, dictLength, blockLength);
buffer = ArrayUtil.grow(buffer, dictLength + blockLength);
bytes.length = 0;
// Read the dictionary
if (LZ4.decompress(in, dictLength, buffer, 0) != dictLength) {
throw new CorruptIndexException("Illegal dict length", in);
}
int offsetInBlock = dictLength;
int offsetInBytesRef = offset;
if (offset >= dictLength) {
offsetInBytesRef -= dictLength;
// Skip unneeded blocks
int numBytesToSkip = 0;
for (int i = 0; i < numBlocks && offsetInBlock + blockLength < offset; ++i) {
int compressedBlockLength = compressedLengths[i];
numBytesToSkip += compressedBlockLength;
offsetInBlock += blockLength;
offsetInBytesRef -= blockLength;
}
in.skipBytes(numBytesToSkip);
} else {
// The dictionary contains some bytes we need, copy its content to the BytesRef
bytes.bytes = ArrayUtil.grow(bytes.bytes, dictLength);
System.arraycopy(buffer, 0, bytes.bytes, 0, dictLength);
bytes.length = dictLength;
}
// Read blocks that intersect with the interval we need
while (offsetInBlock < offset + length) {
final int bytesToDecompress = Math.min(blockLength, offset + length - offsetInBlock);
LZ4.decompress(in, bytesToDecompress, buffer, dictLength);
bytes.bytes = ArrayUtil.grow(bytes.bytes, bytes.length + bytesToDecompress);
System.arraycopy(buffer, dictLength, bytes.bytes, bytes.length, bytesToDecompress);
bytes.length += bytesToDecompress;
offsetInBlock += blockLength;
}
bytes.offset = offsetInBytesRef;
bytes.length = length;
assert bytes.isValid();
}
@Override
public Decompressor clone() {
return new LZ4WithPresetDictDecompressor();
}
}
private static class LZ4WithPresetDictCompressor extends Compressor {
final ByteBuffersDataOutput compressed;
final LZ4.FastCompressionHashTable hashTable;
byte[] buffer;
LZ4WithPresetDictCompressor() {
compressed = ByteBuffersDataOutput.newResettableInstance();
hashTable = new LZ4.FastCompressionHashTable();
buffer = BytesRef.EMPTY_BYTES;
}
private void doCompress(byte[] bytes, int dictLen, int len, DataOutput out) throws IOException {
long prevCompressedSize = compressed.size();
LZ4.compressWithDictionary(bytes, 0, dictLen, len, compressed, hashTable);
// Write the number of compressed bytes
out.writeVInt(Math.toIntExact(compressed.size() - prevCompressedSize));
}
@Override
public void compress(byte[] bytes, int off, int len, DataOutput out) throws IOException {
final int dictLength = len / (NUM_SUB_BLOCKS * DICT_SIZE_FACTOR);
final int blockLength = (len - dictLength + NUM_SUB_BLOCKS - 1) / NUM_SUB_BLOCKS;
buffer = ArrayUtil.grow(buffer, dictLength + blockLength);
out.writeVInt(dictLength);
out.writeVInt(blockLength);
final int end = off + len;
compressed.reset();
// Compress the dictionary first
System.arraycopy(bytes, off, buffer, 0, dictLength);
doCompress(buffer, 0, dictLength, out);
// And then sub blocks
for (int start = off + dictLength; start < end; start += blockLength) {
int l = Math.min(blockLength, off + len - start);
System.arraycopy(bytes, start, buffer, dictLength, l);
doCompress(buffer, dictLength, l, out);
}
// We only wrote lengths so far, now write compressed data
compressed.copyTo(out);
}
@Override
public void close() throws IOException {
// no-op
}
}
}

View File

@ -40,7 +40,6 @@ import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.VectorFormat;
import org.apache.lucene.codecs.lucene86.Lucene86PointsFormat;
import org.apache.lucene.codecs.lucene86.Lucene86SegmentInfoFormat;
import org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
@ -64,8 +63,10 @@ public class Lucene87Codec extends Codec {
Lucene87StoredFieldsFormat.Mode.BEST_COMPRESSION,
Lucene80DocValuesFormat.Mode.BEST_COMPRESSION);
private final Lucene87StoredFieldsFormat.Mode storedMode;
private final Lucene80DocValuesFormat.Mode dvMode;
/** compression mode for stored fields */
protected final Lucene87StoredFieldsFormat.Mode storedMode;
/** compression mode for doc value fields */
protected final Lucene80DocValuesFormat.Mode dvMode;
private Mode(Lucene87StoredFieldsFormat.Mode storedMode, Lucene80DocValuesFormat.Mode dvMode) {
this.storedMode = Objects.requireNonNull(storedMode);
@ -101,14 +102,23 @@ public class Lucene87Codec extends Codec {
/** Instantiates a new codec. */
public Lucene87Codec() {
this(Mode.BEST_COMPRESSION);
}
/**
* Instantiates a new codec, specifying the compression mode to use.
*
* @param mode compression mode to use for newly flushed/merged segments.
*/
public Lucene87Codec(Mode mode) {
super("Lucene87");
this.storedFieldsFormat = new Lucene87StoredFieldsFormat();
this.storedFieldsFormat = new Lucene87StoredFieldsFormat(mode.storedMode);
this.defaultFormat = new Lucene84PostingsFormat();
this.defaultDVFormat = new Lucene80DocValuesFormat();
this.defaultDVFormat = new Lucene80DocValuesFormat(mode.dvMode);
}
@Override
public final StoredFieldsFormat storedFieldsFormat() {
public StoredFieldsFormat storedFieldsFormat() {
return storedFieldsFormat;
}

View File

@ -14,14 +14,14 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene87;
package org.apache.lucene.backward_codecs.lucene87;
import java.io.IOException;
import java.util.Objects;
import org.apache.lucene.backward_codecs.lucene50.compressing.Lucene50CompressingStoredFieldsFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.StoredFieldsWriter;
import org.apache.lucene.codecs.compressing.CompressingStoredFieldsFormat;
import org.apache.lucene.codecs.compressing.CompressionMode;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.SegmentInfo;
@ -139,28 +139,16 @@ public class Lucene87StoredFieldsFormat extends StoredFieldsFormat {
@Override
public StoredFieldsWriter fieldsWriter(Directory directory, SegmentInfo si, IOContext context)
throws IOException {
String previous = si.putAttribute(MODE_KEY, mode.name());
if (previous != null && previous.equals(mode.name()) == false) {
throw new IllegalStateException(
"found existing value for "
+ MODE_KEY
+ " for segment: "
+ si.name
+ "old="
+ previous
+ ", new="
+ mode.name());
}
return impl(mode).fieldsWriter(directory, si, context);
throw new UnsupportedOperationException("Old codecs may only be used for reading");
}
StoredFieldsFormat impl(Mode mode) {
switch (mode) {
case BEST_SPEED:
return new CompressingStoredFieldsFormat(
return new Lucene50CompressingStoredFieldsFormat(
"Lucene87StoredFieldsFastData", BEST_SPEED_MODE, BEST_SPEED_BLOCK_LENGTH, 1024, 10);
case BEST_COMPRESSION:
return new CompressingStoredFieldsFormat(
return new Lucene50CompressingStoredFieldsFormat(
"Lucene87StoredFieldsHighData",
BEST_COMPRESSION_MODE,
BEST_COMPRESSION_BLOCK_LENGTH,
@ -172,14 +160,16 @@ public class Lucene87StoredFieldsFormat extends StoredFieldsFormat {
}
// Shoot for 10 sub blocks of 48kB each.
private static final int BEST_COMPRESSION_BLOCK_LENGTH = 10 * 48 * 1024;
/** Block length for {@link Mode#BEST_COMPRESSION} */
protected static final int BEST_COMPRESSION_BLOCK_LENGTH = 10 * 48 * 1024;
/** Compression mode for {@link Mode#BEST_COMPRESSION} */
public static final CompressionMode BEST_COMPRESSION_MODE =
new DeflateWithPresetDictCompressionMode();
// Shoot for 10 sub blocks of 60kB each.
private static final int BEST_SPEED_BLOCK_LENGTH = 10 * 60 * 1024;
/** Block length for {@link Mode#BEST_SPEED} */
protected static final int BEST_SPEED_BLOCK_LENGTH = 10 * 60 * 1024;
/** Compression mode for {@link Mode#BEST_SPEED} */
public static final CompressionMode BEST_SPEED_MODE = new LZ4WithPresetDictCompressionMode();

View File

@ -17,7 +17,10 @@
package org.apache.lucene.backward_codecs.lucene50;
import java.io.IOException;
import org.apache.lucene.backward_codecs.lucene50.compressing.Lucene50RWCompressingStoredFieldsFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.StoredFieldsWriter;
import org.apache.lucene.codecs.compressing.CompressionMode;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
@ -52,4 +55,18 @@ public final class Lucene50RWStoredFieldsFormat extends Lucene50StoredFieldsForm
}
return impl(mode).fieldsWriter(directory, si, context);
}
@Override
StoredFieldsFormat impl(Mode mode) {
switch (mode) {
case BEST_SPEED:
return new Lucene50RWCompressingStoredFieldsFormat(
"Lucene50StoredFieldsFastData", CompressionMode.FAST, 1 << 14, 128, 10);
case BEST_COMPRESSION:
return new Lucene50RWCompressingStoredFieldsFormat(
"Lucene50StoredFieldsHighData", CompressionMode.HIGH_COMPRESSION, 61440, 512, 10);
default:
throw new AssertionError();
}
}
}

View File

@ -0,0 +1,736 @@
/*
* 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.backward_codecs.lucene50.compressing;
import static org.apache.lucene.backward_codecs.lucene50.compressing.Lucene50CompressingStoredFieldsReader.BYTE_ARR;
import static org.apache.lucene.backward_codecs.lucene50.compressing.Lucene50CompressingStoredFieldsReader.FIELDS_EXTENSION;
import static org.apache.lucene.backward_codecs.lucene50.compressing.Lucene50CompressingStoredFieldsReader.INDEX_CODEC_NAME;
import static org.apache.lucene.backward_codecs.lucene50.compressing.Lucene50CompressingStoredFieldsReader.INDEX_EXTENSION;
import static org.apache.lucene.backward_codecs.lucene50.compressing.Lucene50CompressingStoredFieldsReader.META_EXTENSION;
import static org.apache.lucene.backward_codecs.lucene50.compressing.Lucene50CompressingStoredFieldsReader.NUMERIC_DOUBLE;
import static org.apache.lucene.backward_codecs.lucene50.compressing.Lucene50CompressingStoredFieldsReader.NUMERIC_FLOAT;
import static org.apache.lucene.backward_codecs.lucene50.compressing.Lucene50CompressingStoredFieldsReader.NUMERIC_INT;
import static org.apache.lucene.backward_codecs.lucene50.compressing.Lucene50CompressingStoredFieldsReader.NUMERIC_LONG;
import static org.apache.lucene.backward_codecs.lucene50.compressing.Lucene50CompressingStoredFieldsReader.STRING;
import static org.apache.lucene.backward_codecs.lucene50.compressing.Lucene50CompressingStoredFieldsReader.TYPE_BITS;
import static org.apache.lucene.backward_codecs.lucene50.compressing.Lucene50CompressingStoredFieldsReader.VERSION_CURRENT;
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.StoredFieldsWriter;
import org.apache.lucene.codecs.compressing.CompressionMode;
import org.apache.lucene.codecs.compressing.Compressor;
import org.apache.lucene.codecs.compressing.MatchingReaders;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.DocIDMerger;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BitUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.packed.PackedInts;
/**
* {@link StoredFieldsWriter} impl for {@link Lucene50CompressingStoredFieldsFormat}.
*
* @lucene.experimental
*/
public final class Lucene50CompressingStoredFieldsWriter extends StoredFieldsWriter {
private final String segment;
private FieldsIndexWriter indexWriter;
private IndexOutput metaStream, fieldsStream;
private Compressor compressor;
private final CompressionMode compressionMode;
private final int chunkSize;
private final int maxDocsPerChunk;
private final ByteBuffersDataOutput bufferedDocs;
private int[] numStoredFields; // number of stored fields
private int[] endOffsets; // end offsets in bufferedDocs
private int docBase; // doc ID at the beginning of the chunk
private int numBufferedDocs; // docBase + numBufferedDocs == current doc ID
private long numDirtyChunks; // number of incomplete compressed blocks written
private long numDirtyDocs; // cumulative number of missing docs in incomplete chunks
/** Sole constructor. */
Lucene50CompressingStoredFieldsWriter(
Directory directory,
SegmentInfo si,
String segmentSuffix,
IOContext context,
String formatName,
CompressionMode compressionMode,
int chunkSize,
int maxDocsPerChunk,
int blockShift)
throws IOException {
assert directory != null;
this.segment = si.name;
this.compressionMode = compressionMode;
this.compressor = compressionMode.newCompressor();
this.chunkSize = chunkSize;
this.maxDocsPerChunk = maxDocsPerChunk;
this.docBase = 0;
this.bufferedDocs = ByteBuffersDataOutput.newResettableInstance();
this.numStoredFields = new int[16];
this.endOffsets = new int[16];
this.numBufferedDocs = 0;
boolean success = false;
try {
metaStream =
directory.createOutput(
IndexFileNames.segmentFileName(segment, segmentSuffix, META_EXTENSION), context);
CodecUtil.writeIndexHeader(
metaStream, INDEX_CODEC_NAME + "Meta", VERSION_CURRENT, si.getId(), segmentSuffix);
assert CodecUtil.indexHeaderLength(INDEX_CODEC_NAME + "Meta", segmentSuffix)
== metaStream.getFilePointer();
fieldsStream =
directory.createOutput(
IndexFileNames.segmentFileName(segment, segmentSuffix, FIELDS_EXTENSION), context);
CodecUtil.writeIndexHeader(
fieldsStream, formatName, VERSION_CURRENT, si.getId(), segmentSuffix);
assert CodecUtil.indexHeaderLength(formatName, segmentSuffix)
== fieldsStream.getFilePointer();
indexWriter =
new FieldsIndexWriter(
directory,
segment,
segmentSuffix,
INDEX_EXTENSION,
INDEX_CODEC_NAME,
si.getId(),
blockShift,
context);
metaStream.writeVInt(chunkSize);
metaStream.writeVInt(PackedInts.VERSION_CURRENT);
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(metaStream, fieldsStream, indexWriter);
}
}
}
@Override
public void close() throws IOException {
try {
IOUtils.close(metaStream, fieldsStream, indexWriter, compressor);
} finally {
metaStream = null;
fieldsStream = null;
indexWriter = null;
compressor = null;
}
}
private int numStoredFieldsInDoc;
@Override
public void startDocument() throws IOException {}
@Override
public void finishDocument() throws IOException {
if (numBufferedDocs == this.numStoredFields.length) {
final int newLength = ArrayUtil.oversize(numBufferedDocs + 1, 4);
this.numStoredFields = ArrayUtil.growExact(this.numStoredFields, newLength);
endOffsets = ArrayUtil.growExact(endOffsets, newLength);
}
this.numStoredFields[numBufferedDocs] = numStoredFieldsInDoc;
numStoredFieldsInDoc = 0;
endOffsets[numBufferedDocs] = Math.toIntExact(bufferedDocs.size());
++numBufferedDocs;
if (triggerFlush()) {
flush();
}
}
private static void saveInts(int[] values, int length, DataOutput out) throws IOException {
assert length > 0;
if (length == 1) {
out.writeVInt(values[0]);
} else {
boolean allEqual = true;
for (int i = 1; i < length; ++i) {
if (values[i] != values[0]) {
allEqual = false;
break;
}
}
if (allEqual) {
out.writeVInt(0);
out.writeVInt(values[0]);
} else {
long max = 0;
for (int i = 0; i < length; ++i) {
max |= values[i];
}
final int bitsRequired = PackedInts.bitsRequired(max);
out.writeVInt(bitsRequired);
final PackedInts.Writer w =
PackedInts.getWriterNoHeader(out, PackedInts.Format.PACKED, length, bitsRequired, 1);
for (int i = 0; i < length; ++i) {
w.add(values[i]);
}
w.finish();
}
}
}
private void writeHeader(
int docBase, int numBufferedDocs, int[] numStoredFields, int[] lengths, boolean sliced)
throws IOException {
final int slicedBit = sliced ? 1 : 0;
// save docBase and numBufferedDocs
fieldsStream.writeVInt(docBase);
fieldsStream.writeVInt((numBufferedDocs) << 1 | slicedBit);
// save numStoredFields
saveInts(numStoredFields, numBufferedDocs, fieldsStream);
// save lengths
saveInts(lengths, numBufferedDocs, fieldsStream);
}
private boolean triggerFlush() {
return bufferedDocs.size() >= chunkSize
|| // chunks of at least chunkSize bytes
numBufferedDocs >= maxDocsPerChunk;
}
private void flush() throws IOException {
indexWriter.writeIndex(numBufferedDocs, fieldsStream.getFilePointer());
// transform end offsets into lengths
final int[] lengths = endOffsets;
for (int i = numBufferedDocs - 1; i > 0; --i) {
lengths[i] = endOffsets[i] - endOffsets[i - 1];
assert lengths[i] >= 0;
}
final boolean sliced = bufferedDocs.size() >= 2 * chunkSize;
writeHeader(docBase, numBufferedDocs, numStoredFields, lengths, sliced);
// compress stored fields to fieldsStream.
//
// TODO: do we need to slice it since we already have the slices in the buffer? Perhaps
// we should use max-block-bits restriction on the buffer itself, then we won't have to check it
// here.
byte[] content = bufferedDocs.toArrayCopy();
bufferedDocs.reset();
if (sliced) {
// big chunk, slice it
for (int compressed = 0; compressed < content.length; compressed += chunkSize) {
compressor.compress(
content, compressed, Math.min(chunkSize, content.length - compressed), fieldsStream);
}
} else {
compressor.compress(content, 0, content.length, fieldsStream);
}
// reset
docBase += numBufferedDocs;
numBufferedDocs = 0;
bufferedDocs.reset();
}
@Override
public void writeField(FieldInfo info, IndexableField field) throws IOException {
++numStoredFieldsInDoc;
int bits = 0;
final BytesRef bytes;
final String string;
Number number = field.numericValue();
if (number != null) {
if (number instanceof Byte || number instanceof Short || number instanceof Integer) {
bits = NUMERIC_INT;
} else if (number instanceof Long) {
bits = NUMERIC_LONG;
} else if (number instanceof Float) {
bits = NUMERIC_FLOAT;
} else if (number instanceof Double) {
bits = NUMERIC_DOUBLE;
} else {
throw new IllegalArgumentException("cannot store numeric type " + number.getClass());
}
string = null;
bytes = null;
} else {
bytes = field.binaryValue();
if (bytes != null) {
bits = BYTE_ARR;
string = null;
} else {
bits = STRING;
string = field.stringValue();
if (string == null) {
throw new IllegalArgumentException(
"field "
+ field.name()
+ " is stored but does not have binaryValue, stringValue nor numericValue");
}
}
}
final long infoAndBits = (((long) info.number) << TYPE_BITS) | bits;
bufferedDocs.writeVLong(infoAndBits);
if (bytes != null) {
bufferedDocs.writeVInt(bytes.length);
bufferedDocs.writeBytes(bytes.bytes, bytes.offset, bytes.length);
} else if (string != null) {
bufferedDocs.writeString(string);
} else {
if (number instanceof Byte || number instanceof Short || number instanceof Integer) {
bufferedDocs.writeZInt(number.intValue());
} else if (number instanceof Long) {
writeTLong(bufferedDocs, number.longValue());
} else if (number instanceof Float) {
writeZFloat(bufferedDocs, number.floatValue());
} else if (number instanceof Double) {
writeZDouble(bufferedDocs, number.doubleValue());
} else {
throw new AssertionError("Cannot get here");
}
}
}
// -0 isn't compressed.
static final int NEGATIVE_ZERO_FLOAT = Float.floatToIntBits(-0f);
static final long NEGATIVE_ZERO_DOUBLE = Double.doubleToLongBits(-0d);
// for compression of timestamps
static final long SECOND = 1000L;
static final long HOUR = 60 * 60 * SECOND;
static final long DAY = 24 * HOUR;
static final int SECOND_ENCODING = 0x40;
static final int HOUR_ENCODING = 0x80;
static final int DAY_ENCODING = 0xC0;
/**
* Writes a float in a variable-length format. Writes between one and five bytes. Small integral
* values typically take fewer bytes.
*
* <p>ZFloat --&gt; Header, Bytes*?
*
* <ul>
* <li>Header --&gt; {@link DataOutput#writeByte Uint8}. When it is equal to 0xFF then the value
* is negative and stored in the next 4 bytes. Otherwise if the first bit is set then the
* other bits in the header encode the value plus one and no other bytes are read.
* Otherwise, the value is a positive float value whose first byte is the header, and 3
* bytes need to be read to complete it.
* <li>Bytes --&gt; Potential additional bytes to read depending on the header.
* </ul>
*/
static void writeZFloat(DataOutput out, float f) throws IOException {
int intVal = (int) f;
final int floatBits = Float.floatToIntBits(f);
if (f == intVal && intVal >= -1 && intVal <= 0x7D && floatBits != NEGATIVE_ZERO_FLOAT) {
// small integer value [-1..125]: single byte
out.writeByte((byte) (0x80 | (1 + intVal)));
} else if ((floatBits >>> 31) == 0) {
// other positive floats: 4 bytes
out.writeInt(floatBits);
} else {
// other negative float: 5 bytes
out.writeByte((byte) 0xFF);
out.writeInt(floatBits);
}
}
/**
* Writes a float in a variable-length format. Writes between one and five bytes. Small integral
* values typically take fewer bytes.
*
* <p>ZFloat --&gt; Header, Bytes*?
*
* <ul>
* <li>Header --&gt; {@link DataOutput#writeByte Uint8}. When it is equal to 0xFF then the value
* is negative and stored in the next 8 bytes. When it is equal to 0xFE then the value is
* stored as a float in the next 4 bytes. Otherwise if the first bit is set then the other
* bits in the header encode the value plus one and no other bytes are read. Otherwise, the
* value is a positive float value whose first byte is the header, and 7 bytes need to be
* read to complete it.
* <li>Bytes --&gt; Potential additional bytes to read depending on the header.
* </ul>
*/
static void writeZDouble(DataOutput out, double d) throws IOException {
int intVal = (int) d;
final long doubleBits = Double.doubleToLongBits(d);
if (d == intVal && intVal >= -1 && intVal <= 0x7C && doubleBits != NEGATIVE_ZERO_DOUBLE) {
// small integer value [-1..124]: single byte
out.writeByte((byte) (0x80 | (intVal + 1)));
return;
} else if (d == (float) d) {
// d has an accurate float representation: 5 bytes
out.writeByte((byte) 0xFE);
out.writeInt(Float.floatToIntBits((float) d));
} else if ((doubleBits >>> 63) == 0) {
// other positive doubles: 8 bytes
out.writeLong(doubleBits);
} else {
// other negative doubles: 9 bytes
out.writeByte((byte) 0xFF);
out.writeLong(doubleBits);
}
}
/**
* Writes a long in a variable-length format. Writes between one and ten bytes. Small values or
* values representing timestamps with day, hour or second precision typically require fewer
* bytes.
*
* <p>ZLong --&gt; Header, Bytes*?
*
* <ul>
* <li>Header --&gt; The first two bits indicate the compression scheme:
* <ul>
* <li>00 - uncompressed
* <li>01 - multiple of 1000 (second)
* <li>10 - multiple of 3600000 (hour)
* <li>11 - multiple of 86400000 (day)
* </ul>
* Then the next bit is a continuation bit, indicating whether more bytes need to be read,
* and the last 5 bits are the lower bits of the encoded value. In order to reconstruct the
* value, you need to combine the 5 lower bits of the header with a vLong in the next bytes
* (if the continuation bit is set to 1). Then {@link BitUtil#zigZagDecode(int)
* zigzag-decode} it and finally multiply by the multiple corresponding to the compression
* scheme.
* <li>Bytes --&gt; Potential additional bytes to read depending on the header.
* </ul>
*/
// T for "timestamp"
static void writeTLong(DataOutput out, long l) throws IOException {
int header;
if (l % SECOND != 0) {
header = 0;
} else if (l % DAY == 0) {
// timestamp with day precision
header = DAY_ENCODING;
l /= DAY;
} else if (l % HOUR == 0) {
// timestamp with hour precision, or day precision with a timezone
header = HOUR_ENCODING;
l /= HOUR;
} else {
// timestamp with second precision
header = SECOND_ENCODING;
l /= SECOND;
}
final long zigZagL = BitUtil.zigZagEncode(l);
header |= (zigZagL & 0x1F); // last 5 bits
final long upperBits = zigZagL >>> 5;
if (upperBits != 0) {
header |= 0x20;
}
out.writeByte((byte) header);
if (upperBits != 0) {
out.writeVLong(upperBits);
}
}
@Override
public void finish(FieldInfos fis, int numDocs) throws IOException {
if (numBufferedDocs > 0) {
numDirtyChunks++; // incomplete: we had to force this flush
final long expectedChunkDocs =
Math.min(
maxDocsPerChunk, (long) ((double) chunkSize / bufferedDocs.size() * numBufferedDocs));
numDirtyDocs += expectedChunkDocs - numBufferedDocs;
flush();
} else {
assert bufferedDocs.size() == 0;
}
if (docBase != numDocs) {
throw new RuntimeException(
"Wrote " + docBase + " docs, finish called with numDocs=" + numDocs);
}
indexWriter.finish(numDocs, fieldsStream.getFilePointer(), metaStream);
metaStream.writeVLong(numDirtyChunks);
metaStream.writeVLong(numDirtyDocs);
CodecUtil.writeFooter(metaStream);
CodecUtil.writeFooter(fieldsStream);
assert bufferedDocs.size() == 0;
}
// bulk merge is scary: its caused corruption bugs in the past.
// we try to be extra safe with this impl, but add an escape hatch to
// have a workaround for undiscovered bugs.
static final String BULK_MERGE_ENABLED_SYSPROP =
Lucene50CompressingStoredFieldsWriter.class.getName() + ".enableBulkMerge";
static final boolean BULK_MERGE_ENABLED;
static {
boolean v = true;
try {
v = Boolean.parseBoolean(System.getProperty(BULK_MERGE_ENABLED_SYSPROP, "true"));
} catch (SecurityException ignored) {
}
BULK_MERGE_ENABLED = v;
}
@Override
public int merge(MergeState mergeState) throws IOException {
int docCount = 0;
int numReaders = mergeState.maxDocs.length;
MatchingReaders matching = new MatchingReaders(mergeState);
if (mergeState.needsIndexSort) {
/**
* If all readers are compressed and they have the same fieldinfos then we can merge the
* serialized document directly.
*/
List<CompressingStoredFieldsMergeSub> subs = new ArrayList<>();
for (int i = 0; i < mergeState.storedFieldsReaders.length; i++) {
if (matching.matchingReaders[i]
&& mergeState.storedFieldsReaders[i] instanceof Lucene50CompressingStoredFieldsReader) {
Lucene50CompressingStoredFieldsReader storedFieldsReader =
(Lucene50CompressingStoredFieldsReader) mergeState.storedFieldsReaders[i];
storedFieldsReader.checkIntegrity();
subs.add(
new CompressingStoredFieldsMergeSub(
storedFieldsReader, mergeState.docMaps[i], mergeState.maxDocs[i]));
} else {
return super.merge(mergeState);
}
}
final DocIDMerger<CompressingStoredFieldsMergeSub> docIDMerger = DocIDMerger.of(subs, true);
while (true) {
CompressingStoredFieldsMergeSub sub = docIDMerger.next();
if (sub == null) {
break;
}
assert sub.mappedDocID == docCount;
Lucene50CompressingStoredFieldsReader.SerializedDocument doc =
sub.reader.document(sub.docID);
startDocument();
bufferedDocs.copyBytes(doc.in, doc.length);
numStoredFieldsInDoc = doc.numStoredFields;
finishDocument();
++docCount;
}
finish(mergeState.mergeFieldInfos, docCount);
return docCount;
}
for (int readerIndex = 0; readerIndex < numReaders; readerIndex++) {
MergeVisitor visitor = new MergeVisitor(mergeState, readerIndex);
Lucene50CompressingStoredFieldsReader matchingFieldsReader = null;
if (matching.matchingReaders[readerIndex]) {
final StoredFieldsReader fieldsReader = mergeState.storedFieldsReaders[readerIndex];
// we can only bulk-copy if the matching reader is also a CompressingStoredFieldsReader
if (fieldsReader != null && fieldsReader instanceof Lucene50CompressingStoredFieldsReader) {
matchingFieldsReader = (Lucene50CompressingStoredFieldsReader) fieldsReader;
}
}
final int maxDoc = mergeState.maxDocs[readerIndex];
final Bits liveDocs = mergeState.liveDocs[readerIndex];
// if its some other format, or an older version of this format, or safety switch:
if (matchingFieldsReader == null
|| matchingFieldsReader.getVersion() != VERSION_CURRENT
|| BULK_MERGE_ENABLED == false) {
// naive merge...
StoredFieldsReader storedFieldsReader = mergeState.storedFieldsReaders[readerIndex];
if (storedFieldsReader != null) {
storedFieldsReader.checkIntegrity();
}
for (int docID = 0; docID < maxDoc; docID++) {
if (liveDocs != null && liveDocs.get(docID) == false) {
continue;
}
startDocument();
storedFieldsReader.visitDocument(docID, visitor);
finishDocument();
++docCount;
}
} else if (matchingFieldsReader.getCompressionMode() == compressionMode
&& matchingFieldsReader.getChunkSize() == chunkSize
&& matchingFieldsReader.getPackedIntsVersion() == PackedInts.VERSION_CURRENT
&& liveDocs == null
&& !tooDirty(matchingFieldsReader)) {
// optimized merge, raw byte copy
// its not worth fine-graining this if there are deletions.
// if the format is older, its always handled by the naive merge case above
assert matchingFieldsReader.getVersion() == VERSION_CURRENT;
matchingFieldsReader.checkIntegrity();
// flush any pending chunks
if (numBufferedDocs > 0) {
flush();
numDirtyChunks++; // incomplete: we had to force this flush
}
// iterate over each chunk. we use the stored fields index to find chunk boundaries,
// read the docstart + doccount from the chunk header (we write a new header, since doc
// numbers will change),
// and just copy the bytes directly.
IndexInput rawDocs = matchingFieldsReader.getFieldsStream();
FieldsIndex index = matchingFieldsReader.getIndexReader();
rawDocs.seek(index.getStartPointer(0));
int docID = 0;
while (docID < maxDoc) {
// read header
int base = rawDocs.readVInt();
if (base != docID) {
throw new CorruptIndexException(
"invalid state: base=" + base + ", docID=" + docID, rawDocs);
}
int code = rawDocs.readVInt();
// write a new index entry and new header for this chunk.
int bufferedDocs = code >>> 1;
indexWriter.writeIndex(bufferedDocs, fieldsStream.getFilePointer());
fieldsStream.writeVInt(docBase); // rebase
fieldsStream.writeVInt(code);
docID += bufferedDocs;
docBase += bufferedDocs;
docCount += bufferedDocs;
if (docID > maxDoc) {
throw new CorruptIndexException(
"invalid state: base=" + base + ", count=" + bufferedDocs + ", maxDoc=" + maxDoc,
rawDocs);
}
// copy bytes until the next chunk boundary (or end of chunk data).
// using the stored fields index for this isn't the most efficient, but fast enough
// and is a source of redundancy for detecting bad things.
final long end;
if (docID == maxDoc) {
end = matchingFieldsReader.getMaxPointer();
} else {
end = index.getStartPointer(docID);
}
fieldsStream.copyBytes(rawDocs, end - rawDocs.getFilePointer());
}
if (rawDocs.getFilePointer() != matchingFieldsReader.getMaxPointer()) {
throw new CorruptIndexException(
"invalid state: pos="
+ rawDocs.getFilePointer()
+ ", max="
+ matchingFieldsReader.getMaxPointer(),
rawDocs);
}
// since we bulk merged all chunks, we inherit any dirty ones from this segment.
numDirtyChunks += matchingFieldsReader.getNumDirtyChunks();
numDirtyDocs += matchingFieldsReader.getNumDirtyDocs();
} else {
// optimized merge, we copy serialized (but decompressed) bytes directly
// even on simple docs (1 stored field), it seems to help by about 20%
// if the format is older, its always handled by the naive merge case above
assert matchingFieldsReader.getVersion() == VERSION_CURRENT;
matchingFieldsReader.checkIntegrity();
for (int docID = 0; docID < maxDoc; docID++) {
if (liveDocs != null && liveDocs.get(docID) == false) {
continue;
}
Lucene50CompressingStoredFieldsReader.SerializedDocument doc =
matchingFieldsReader.document(docID);
startDocument();
bufferedDocs.copyBytes(doc.in, doc.length);
numStoredFieldsInDoc = doc.numStoredFields;
finishDocument();
++docCount;
}
}
}
finish(mergeState.mergeFieldInfos, docCount);
return docCount;
}
/**
* Returns true if we should recompress this reader, even though we could bulk merge compressed
* data
*
* <p>The last chunk written for a segment is typically incomplete, so without recompressing, in
* some worst-case situations (e.g. frequent reopen with tiny flushes), over time the compression
* ratio can degrade. This is a safety switch.
*/
boolean tooDirty(Lucene50CompressingStoredFieldsReader candidate) {
// more than 1% dirty, or more than hard limit of 1024 dirty chunks
return candidate.getNumDirtyChunks() > 1024
|| candidate.getNumDirtyDocs() * 100 > candidate.getNumDocs();
}
private static class CompressingStoredFieldsMergeSub extends DocIDMerger.Sub {
private final Lucene50CompressingStoredFieldsReader reader;
private final int maxDoc;
int docID = -1;
CompressingStoredFieldsMergeSub(
Lucene50CompressingStoredFieldsReader reader, MergeState.DocMap docMap, int maxDoc) {
super(docMap);
this.maxDoc = maxDoc;
this.reader = reader;
}
@Override
public int nextDoc() {
docID++;
if (docID == maxDoc) {
return NO_MORE_DOCS;
} else {
return docID;
}
}
}
@Override
public long ramBytesUsed() {
return bufferedDocs.ramBytesUsed()
+ numStoredFields.length * Integer.BYTES
+ endOffsets.length * Integer.BYTES;
}
}

View File

@ -0,0 +1,52 @@
/*
* 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.backward_codecs.lucene50.compressing;
import java.io.IOException;
import org.apache.lucene.codecs.StoredFieldsWriter;
import org.apache.lucene.codecs.compressing.CompressionMode;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
/** RW impersonation of Lucene50CompressingStoredFieldsFormat. */
public class Lucene50RWCompressingStoredFieldsFormat extends Lucene50CompressingStoredFieldsFormat {
public Lucene50RWCompressingStoredFieldsFormat(
String formatName,
CompressionMode compressionMode,
int chunkSize,
int maxDocsPerChunk,
int blockShift) {
super(formatName, compressionMode, chunkSize, maxDocsPerChunk, blockShift);
}
@Override
public StoredFieldsWriter fieldsWriter(Directory directory, SegmentInfo si, IOContext context)
throws IOException {
return new Lucene50CompressingStoredFieldsWriter(
directory,
si,
segmentSuffix,
context,
formatName,
compressionMode,
chunkSize,
maxDocsPerChunk,
blockShift);
}
}

View File

@ -23,6 +23,7 @@ import org.apache.lucene.backward_codecs.lucene84.Lucene84RWPostingsFormat;
import org.apache.lucene.codecs.CompoundFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
@ -37,6 +38,16 @@ public class Lucene87RWCodec extends Lucene87Codec {
return defaultPF;
}
};
private final Mode mode;
public Lucene87RWCodec() {
this(Mode.BEST_SPEED);
}
public Lucene87RWCodec(Mode mode) {
super(mode);
this.mode = mode;
}
@Override
public final CompoundFormat compoundFormat() {
@ -57,4 +68,9 @@ public class Lucene87RWCodec extends Lucene87Codec {
public TermVectorsFormat termVectorsFormat() {
return new Lucene50RWTermVectorsFormat();
}
@Override
public StoredFieldsFormat storedFieldsFormat() {
return new Lucene87RWStoredFieldsFormat(mode.storedMode);
}
}

View File

@ -0,0 +1,71 @@
/*
* 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.backward_codecs.lucene87;
import java.io.IOException;
import org.apache.lucene.backward_codecs.lucene50.compressing.Lucene50RWCompressingStoredFieldsFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.StoredFieldsWriter;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
;
/** RW impersonation of Lucene87StoredFieldsFormat. */
public class Lucene87RWStoredFieldsFormat extends Lucene87StoredFieldsFormat {
public Lucene87RWStoredFieldsFormat(Mode mode) {
super(mode);
}
@Override
public StoredFieldsWriter fieldsWriter(Directory directory, SegmentInfo si, IOContext context)
throws IOException {
String previous = si.putAttribute(MODE_KEY, mode.name());
if (previous != null && previous.equals(mode.name()) == false) {
throw new IllegalStateException(
"found existing value for "
+ MODE_KEY
+ " for segment: "
+ si.name
+ "old="
+ previous
+ ", new="
+ mode.name());
}
return impl(mode).fieldsWriter(directory, si, context);
}
@Override
StoredFieldsFormat impl(Mode mode) {
switch (mode) {
case BEST_SPEED:
return new Lucene50RWCompressingStoredFieldsFormat(
"Lucene87StoredFieldsFastData", BEST_SPEED_MODE, BEST_SPEED_BLOCK_LENGTH, 1024, 10);
case BEST_COMPRESSION:
return new Lucene50RWCompressingStoredFieldsFormat(
"Lucene87StoredFieldsHighData",
BEST_COMPRESSION_MODE,
BEST_COMPRESSION_BLOCK_LENGTH,
4096,
10);
default:
throw new AssertionError();
}
}
}

View File

@ -14,15 +14,14 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene87;
package org.apache.lucene.backward_codecs.lucene87;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.BaseStoredFieldsFormatTestCase;
import org.apache.lucene.util.TestUtil;
public class TestLucene87StoredFieldsFormat extends BaseStoredFieldsFormatTestCase {
@Override
protected Codec getCodec() {
return TestUtil.getDefaultCodec();
return new Lucene87RWCodec();
}
}

View File

@ -14,12 +14,10 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene87;
package org.apache.lucene.backward_codecs.lucene87;
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.lucene90.Lucene90Codec;
import org.apache.lucene.codecs.lucene90.Lucene90Codec.Mode;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.StoredField;
import org.apache.lucene.index.BaseStoredFieldsFormatTestCase;
@ -31,7 +29,7 @@ import org.apache.lucene.store.Directory;
public class TestLucene87StoredFieldsFormatHighCompression extends BaseStoredFieldsFormatTestCase {
@Override
protected Codec getCodec() {
return new Lucene90Codec(Mode.BEST_COMPRESSION);
return new Lucene87RWCodec(Lucene87Codec.Mode.BEST_COMPRESSION);
}
/**
@ -41,7 +39,8 @@ public class TestLucene87StoredFieldsFormatHighCompression extends BaseStoredFie
Directory dir = newDirectory();
for (int i = 0; i < 10; i++) {
IndexWriterConfig iwc = newIndexWriterConfig();
iwc.setCodec(new Lucene90Codec(RandomPicks.randomFrom(random(), Mode.values())));
iwc.setCodec(
new Lucene87RWCodec(RandomPicks.randomFrom(random(), Lucene87Codec.Mode.values())));
IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig());
Document doc = new Document();
doc.add(new StoredField("field1", "value1"));
@ -70,7 +69,7 @@ public class TestLucene87StoredFieldsFormatHighCompression extends BaseStoredFie
expectThrows(
NullPointerException.class,
() -> {
new Lucene90Codec(null);
new Lucene87RWCodec(null);
});
expectThrows(

View File

@ -14,9 +14,9 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene87;
package org.apache.lucene.backward_codecs.lucene87;
/** Test the merge instance of the Lucene50 stored fields format. */
/** Test the merge instance of the Lucene87 stored fields format. */
public class TestLucene87StoredFieldsFormatMergeInstance extends TestLucene87StoredFieldsFormat {
@Override

View File

@ -1,238 +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.compressing;
import static org.apache.lucene.util.BitUtil.zigZagDecode;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.packed.PackedInts;
/**
* Random-access reader for {@link FieldsIndexWriter}.
*
* @lucene.internal
*/
final class LegacyFieldsIndexReader extends FieldsIndex {
private static final long BASE_RAM_BYTES_USED =
RamUsageEstimator.shallowSizeOfInstance(LegacyFieldsIndexReader.class);
final int maxDoc;
final int[] docBases;
final long[] startPointers;
final int[] avgChunkDocs;
final long[] avgChunkSizes;
final PackedInts.Reader[] docBasesDeltas; // delta from the avg
final PackedInts.Reader[] startPointersDeltas; // delta from the avg
// It is the responsibility of the caller to close fieldsIndexIn after this constructor
// has been called
LegacyFieldsIndexReader(IndexInput fieldsIndexIn, SegmentInfo si) throws IOException {
maxDoc = si.maxDoc();
int[] docBases = new int[16];
long[] startPointers = new long[16];
int[] avgChunkDocs = new int[16];
long[] avgChunkSizes = new long[16];
PackedInts.Reader[] docBasesDeltas = new PackedInts.Reader[16];
PackedInts.Reader[] startPointersDeltas = new PackedInts.Reader[16];
final int packedIntsVersion = fieldsIndexIn.readVInt();
int blockCount = 0;
for (; ; ) {
final int numChunks = fieldsIndexIn.readVInt();
if (numChunks == 0) {
break;
}
if (blockCount == docBases.length) {
final int newSize = ArrayUtil.oversize(blockCount + 1, 8);
docBases = ArrayUtil.growExact(docBases, newSize);
startPointers = ArrayUtil.growExact(startPointers, newSize);
avgChunkDocs = ArrayUtil.growExact(avgChunkDocs, newSize);
avgChunkSizes = ArrayUtil.growExact(avgChunkSizes, newSize);
docBasesDeltas = ArrayUtil.growExact(docBasesDeltas, newSize);
startPointersDeltas = ArrayUtil.growExact(startPointersDeltas, newSize);
}
// doc bases
docBases[blockCount] = fieldsIndexIn.readVInt();
avgChunkDocs[blockCount] = fieldsIndexIn.readVInt();
final int bitsPerDocBase = fieldsIndexIn.readVInt();
if (bitsPerDocBase > 32) {
throw new CorruptIndexException(
"Corrupted bitsPerDocBase: " + bitsPerDocBase, fieldsIndexIn);
}
docBasesDeltas[blockCount] =
PackedInts.getReaderNoHeader(
fieldsIndexIn,
PackedInts.Format.PACKED,
packedIntsVersion,
numChunks,
bitsPerDocBase);
// start pointers
startPointers[blockCount] = fieldsIndexIn.readVLong();
avgChunkSizes[blockCount] = fieldsIndexIn.readVLong();
final int bitsPerStartPointer = fieldsIndexIn.readVInt();
if (bitsPerStartPointer > 64) {
throw new CorruptIndexException(
"Corrupted bitsPerStartPointer: " + bitsPerStartPointer, fieldsIndexIn);
}
startPointersDeltas[blockCount] =
PackedInts.getReaderNoHeader(
fieldsIndexIn,
PackedInts.Format.PACKED,
packedIntsVersion,
numChunks,
bitsPerStartPointer);
++blockCount;
}
this.docBases = ArrayUtil.copyOfSubArray(docBases, 0, blockCount);
this.startPointers = ArrayUtil.copyOfSubArray(startPointers, 0, blockCount);
this.avgChunkDocs = ArrayUtil.copyOfSubArray(avgChunkDocs, 0, blockCount);
this.avgChunkSizes = ArrayUtil.copyOfSubArray(avgChunkSizes, 0, blockCount);
this.docBasesDeltas = ArrayUtil.copyOfSubArray(docBasesDeltas, 0, blockCount);
this.startPointersDeltas = ArrayUtil.copyOfSubArray(startPointersDeltas, 0, blockCount);
}
private int block(int docID) {
int lo = 0, hi = docBases.length - 1;
while (lo <= hi) {
final int mid = (lo + hi) >>> 1;
final int midValue = docBases[mid];
if (midValue == docID) {
return mid;
} else if (midValue < docID) {
lo = mid + 1;
} else {
hi = mid - 1;
}
}
return hi;
}
private int relativeDocBase(int block, int relativeChunk) {
final int expected = avgChunkDocs[block] * relativeChunk;
final long delta = zigZagDecode(docBasesDeltas[block].get(relativeChunk));
return expected + (int) delta;
}
private long relativeStartPointer(int block, int relativeChunk) {
final long expected = avgChunkSizes[block] * relativeChunk;
final long delta = zigZagDecode(startPointersDeltas[block].get(relativeChunk));
return expected + delta;
}
private int relativeChunk(int block, int relativeDoc) {
int lo = 0, hi = docBasesDeltas[block].size() - 1;
while (lo <= hi) {
final int mid = (lo + hi) >>> 1;
final int midValue = relativeDocBase(block, mid);
if (midValue == relativeDoc) {
return mid;
} else if (midValue < relativeDoc) {
lo = mid + 1;
} else {
hi = mid - 1;
}
}
return hi;
}
long getStartPointer(int docID) {
if (docID < 0 || docID >= maxDoc) {
throw new IllegalArgumentException("docID out of range [0-" + maxDoc + "]: " + docID);
}
final int block = block(docID);
final int relativeChunk = relativeChunk(block, docID - docBases[block]);
return startPointers[block] + relativeStartPointer(block, relativeChunk);
}
@Override
public LegacyFieldsIndexReader clone() {
return this;
}
@Override
public long ramBytesUsed() {
long res = BASE_RAM_BYTES_USED;
res += RamUsageEstimator.shallowSizeOf(docBasesDeltas);
for (PackedInts.Reader r : docBasesDeltas) {
res += r.ramBytesUsed();
}
res += RamUsageEstimator.shallowSizeOf(startPointersDeltas);
for (PackedInts.Reader r : startPointersDeltas) {
res += r.ramBytesUsed();
}
res += RamUsageEstimator.sizeOf(docBases);
res += RamUsageEstimator.sizeOf(startPointers);
res += RamUsageEstimator.sizeOf(avgChunkDocs);
res += RamUsageEstimator.sizeOf(avgChunkSizes);
return res;
}
@Override
public Collection<Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
long docBaseDeltaBytes = RamUsageEstimator.shallowSizeOf(docBasesDeltas);
for (PackedInts.Reader r : docBasesDeltas) {
docBaseDeltaBytes += r.ramBytesUsed();
}
resources.add(Accountables.namedAccountable("doc base deltas", docBaseDeltaBytes));
long startPointerDeltaBytes = RamUsageEstimator.shallowSizeOf(startPointersDeltas);
for (PackedInts.Reader r : startPointersDeltas) {
startPointerDeltaBytes += r.ramBytesUsed();
}
resources.add(Accountables.namedAccountable("start pointer deltas", startPointerDeltaBytes));
return Collections.unmodifiableList(resources);
}
@Override
public String toString() {
return getClass().getSimpleName() + "(blocks=" + docBases.length + ")";
}
@Override
public void close() throws IOException {
// nothing to do
}
@Override
void checkIntegrity() throws IOException {
// nothing to do, the index is checked at open time
}
}

View File

@ -15,5 +15,5 @@
* limitations under the License.
*/
/** StoredFieldsFormat that allows cross-document and cross-field compression of stored fields. */
/** Compressing helper classes. */
package org.apache.lucene.codecs.compressing;

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene87;
package org.apache.lucene.codecs.lucene90;
import java.util.Arrays;
import java.util.zip.DataFormatException;
@ -26,6 +26,8 @@ import org.apache.lucene.util.SuppressForbidden;
/**
* This class is a workaround for JDK bug <a
* href="https://bugs.openjdk.java.net/browse/JDK-8252739">JDK-8252739</a>.
*
* @lucene.internal
*/
@FunctionalInterface
interface BugfixDeflater_JDK8252739 {

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene87;
package org.apache.lucene.codecs.lucene90;
import java.io.IOException;
import java.util.zip.DataFormatException;

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene87;
package org.apache.lucene.codecs.lucene90;
import java.io.IOException;
import org.apache.lucene.codecs.compressing.CompressionMode;

View File

@ -32,7 +32,6 @@ import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.VectorFormat;
import org.apache.lucene.codecs.lucene86.Lucene86PointsFormat;
import org.apache.lucene.codecs.lucene86.Lucene86SegmentInfoFormat;
import org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
@ -49,16 +48,16 @@ public class Lucene90Codec extends Codec {
/** Configuration option for the codec. */
public static enum Mode {
/** Trade compression ratio for retrieval speed. */
BEST_SPEED(Lucene87StoredFieldsFormat.Mode.BEST_SPEED, Lucene90DocValuesFormat.Mode.BEST_SPEED),
BEST_SPEED(Lucene90StoredFieldsFormat.Mode.BEST_SPEED, Lucene90DocValuesFormat.Mode.BEST_SPEED),
/** Trade retrieval speed for compression ratio. */
BEST_COMPRESSION(
Lucene87StoredFieldsFormat.Mode.BEST_COMPRESSION,
Lucene90StoredFieldsFormat.Mode.BEST_COMPRESSION,
Lucene90DocValuesFormat.Mode.BEST_COMPRESSION);
private final Lucene87StoredFieldsFormat.Mode storedMode;
private final Lucene90StoredFieldsFormat.Mode storedMode;
private final Lucene90DocValuesFormat.Mode dvMode;
private Mode(Lucene87StoredFieldsFormat.Mode storedMode, Lucene90DocValuesFormat.Mode dvMode) {
private Mode(Lucene90StoredFieldsFormat.Mode storedMode, Lucene90DocValuesFormat.Mode dvMode) {
this.storedMode = Objects.requireNonNull(storedMode);
this.dvMode = Objects.requireNonNull(dvMode);
}
@ -104,7 +103,7 @@ public class Lucene90Codec extends Codec {
public Lucene90Codec(Mode mode) {
super("Lucene90");
this.storedFieldsFormat =
new Lucene87StoredFieldsFormat(Objects.requireNonNull(mode).storedMode);
new Lucene90StoredFieldsFormat(Objects.requireNonNull(mode).storedMode);
this.defaultFormat = new Lucene90PostingsFormat();
this.defaultDVFormat = new Lucene90DocValuesFormat(mode.dvMode);
}

View File

@ -0,0 +1,186 @@
/*
* 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.Objects;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.StoredFieldsWriter;
import org.apache.lucene.codecs.compressing.CompressionMode;
import org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsFormat;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.StoredFieldVisitor;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.packed.DirectMonotonicWriter;
/**
* Lucene 8.7 stored fields format.
*
* <p><b>Principle</b>
*
* <p>This {@link StoredFieldsFormat} compresses blocks of documents in order to improve the
* compression ratio compared to document-level compression. It uses the <a
* href="http://code.google.com/p/lz4/">LZ4</a> compression algorithm by default in 16KB blocks,
* which is fast to compress and very fast to decompress data. Although the default compression
* method that is used ({@link Mode#BEST_SPEED BEST_SPEED}) focuses more on speed than on
* compression ratio, it should provide interesting compression ratios for redundant inputs (such as
* log files, HTML or plain text). For higher compression, you can choose ({@link
* Mode#BEST_COMPRESSION BEST_COMPRESSION}), which uses the <a
* href="http://en.wikipedia.org/wiki/DEFLATE">DEFLATE</a> algorithm with 48kB blocks and shared
* dictionaries for a better ratio at the expense of slower performance. These two options can be
* configured like this:
*
* <pre class="prettyprint">
* // the default: for high performance
* indexWriterConfig.setCodec(new Lucene87Codec(Mode.BEST_SPEED));
* // instead for higher performance (but slower):
* // indexWriterConfig.setCodec(new Lucene87Codec(Mode.BEST_COMPRESSION));
* </pre>
*
* <p><b>File formats</b>
*
* <p>Stored fields are represented by three files:
*
* <ol>
* <li><a id="field_data"></a>
* <p>A fields data file (extension <code>.fdt</code>). This file stores a compact
* representation of documents in compressed blocks of 16KB or more. When writing a segment,
* documents are appended to an in-memory <code>byte[]</code> buffer. When its size reaches
* 16KB or more, some metadata about the documents is flushed to disk, immediately followed by
* a compressed representation of the buffer using the <a
* href="https://github.com/lz4/lz4">LZ4</a> <a
* href="http://fastcompression.blogspot.fr/2011/05/lz4-explained.html">compression
* format</a>.
* <p>Notes
* <ul>
* <li>When at least one document in a chunk is large enough so that the chunk is larger
* than 32KB, the chunk will actually be compressed in several LZ4 blocks of 16KB. This
* allows {@link StoredFieldVisitor}s which are only interested in the first fields of a
* document to not have to decompress 10MB of data if the document is 10MB, but only
* 16KB.
* <li>Given that the original lengths are written in the metadata of the chunk, the
* decompressor can leverage this information to stop decoding as soon as enough data
* has been decompressed.
* <li>In case documents are incompressible, the overhead of the compression format is less
* than 0.5%.
* </ul>
* <li><a id="field_index"></a>
* <p>A fields index file (extension <code>.fdx</code>). This file stores two {@link
* DirectMonotonicWriter monotonic arrays}, one for the first doc IDs of each block of
* compressed documents, and another one for the corresponding offsets on disk. At search
* time, the array containing doc IDs is binary-searched in order to find the block that
* contains the expected doc ID, and the associated offset on disk is retrieved from the
* second array.
* <li><a id="field_meta"></a>
* <p>A fields meta file (extension <code>.fdm</code>). This file stores metadata about the
* monotonic arrays stored in the index file.
* </ol>
*
* <p><b>Known limitations</b>
*
* <p>This {@link StoredFieldsFormat} does not support individual documents larger than (<code>
* 2<sup>31</sup> - 2<sup>14</sup></code>) bytes.
*
* @lucene.experimental
*/
public class Lucene90StoredFieldsFormat extends StoredFieldsFormat {
/** Configuration option for stored fields. */
public static 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 = Lucene90StoredFieldsFormat.class.getSimpleName() + ".mode";
final Mode mode;
/** Stored fields format with default options */
public Lucene90StoredFieldsFormat() {
this(Mode.BEST_SPEED);
}
/** Stored fields format with specified mode */
public Lucene90StoredFieldsFormat(Mode mode) {
this.mode = Objects.requireNonNull(mode);
}
@Override
public StoredFieldsReader fieldsReader(
Directory directory, SegmentInfo si, FieldInfos fn, IOContext context) throws IOException {
String value = si.getAttribute(MODE_KEY);
if (value == null) {
throw new IllegalStateException("missing value for " + MODE_KEY + " for segment: " + si.name);
}
Mode mode = Mode.valueOf(value);
return impl(mode).fieldsReader(directory, si, fn, context);
}
@Override
public StoredFieldsWriter fieldsWriter(Directory directory, SegmentInfo si, IOContext context)
throws IOException {
String previous = si.putAttribute(MODE_KEY, mode.name());
if (previous != null && previous.equals(mode.name()) == false) {
throw new IllegalStateException(
"found existing value for "
+ MODE_KEY
+ " for segment: "
+ si.name
+ "old="
+ previous
+ ", new="
+ mode.name());
}
return impl(mode).fieldsWriter(directory, si, context);
}
StoredFieldsFormat impl(Mode mode) {
switch (mode) {
case BEST_SPEED:
return new Lucene90CompressingStoredFieldsFormat(
"Lucene87StoredFieldsFastData", BEST_SPEED_MODE, BEST_SPEED_BLOCK_LENGTH, 1024, 10);
case BEST_COMPRESSION:
return new Lucene90CompressingStoredFieldsFormat(
"Lucene87StoredFieldsHighData",
BEST_COMPRESSION_MODE,
BEST_COMPRESSION_BLOCK_LENGTH,
4096,
10);
default:
throw new AssertionError();
}
}
// Shoot for 10 sub blocks of 48kB each.
private static final int BEST_COMPRESSION_BLOCK_LENGTH = 10 * 48 * 1024;
/** Compression mode for {@link Mode#BEST_COMPRESSION} */
public static final CompressionMode BEST_COMPRESSION_MODE =
new DeflateWithPresetDictCompressionMode();
// Shoot for 10 sub blocks of 60kB each.
private static final int BEST_SPEED_BLOCK_LENGTH = 10 * 60 * 1024;
/** Compression mode for {@link Mode#BEST_SPEED} */
public static final CompressionMode BEST_SPEED_MODE = new LZ4WithPresetDictCompressionMode();
}

View File

@ -19,9 +19,8 @@ package org.apache.lucene.codecs.lucene90;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.compressing.CompressionMode;
import org.apache.lucene.codecs.compressing.FieldsIndexWriter;
import org.apache.lucene.codecs.compressing.Lucene90CompressingTermVectorsFormat;
import org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat;
import org.apache.lucene.codecs.lucene90.compressing.FieldsIndexWriter;
import org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingTermVectorsFormat;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.packed.BlockPackedWriter;
import org.apache.lucene.util.packed.PackedInts;
@ -29,7 +28,7 @@ import org.apache.lucene.util.packed.PackedInts;
/**
* Lucene 9.0 {@link TermVectorsFormat term vectors format}.
*
* <p>Very similarly to {@link Lucene87StoredFieldsFormat}, this format is based on compressed
* <p>Very similarly to {@link Lucene90StoredFieldsFormat}, this format is based on compressed
* chunks of data, with document-level granularity so that a document can never span across distinct
* chunks. Moreover, data is made as compact as possible:
*

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.compressing;
package org.apache.lucene.codecs.lucene90.compressing;
import java.io.Closeable;
import java.io.IOException;

View File

@ -14,10 +14,10 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.compressing;
package org.apache.lucene.codecs.lucene90.compressing;
import static org.apache.lucene.codecs.compressing.FieldsIndexWriter.VERSION_CURRENT;
import static org.apache.lucene.codecs.compressing.FieldsIndexWriter.VERSION_START;
import static org.apache.lucene.codecs.lucene90.compressing.FieldsIndexWriter.VERSION_CURRENT;
import static org.apache.lucene.codecs.lucene90.compressing.FieldsIndexWriter.VERSION_START;
import java.io.IOException;
import java.io.UncheckedIOException;

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.compressing;
package org.apache.lucene.codecs.lucene90.compressing;
import java.io.Closeable;
import java.io.IOException;

View File

@ -14,13 +14,14 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.compressing;
package org.apache.lucene.codecs.lucene90.compressing;
import java.io.IOException;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.StoredFieldsWriter;
import org.apache.lucene.codecs.compressing.CompressionMode;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.MergePolicy;
import org.apache.lucene.index.SegmentInfo;
@ -40,7 +41,7 @@ import org.apache.lucene.util.packed.DirectMonotonicWriter;
*
* @lucene.experimental
*/
public class CompressingStoredFieldsFormat extends StoredFieldsFormat {
public class Lucene90CompressingStoredFieldsFormat extends StoredFieldsFormat {
private final String formatName;
private final String segmentSuffix;
@ -50,12 +51,12 @@ public class CompressingStoredFieldsFormat extends StoredFieldsFormat {
private final int blockShift;
/**
* Create a new {@link CompressingStoredFieldsFormat} with an empty segment suffix.
* Create a new {@link Lucene90CompressingStoredFieldsFormat} with an empty segment suffix.
*
* @see CompressingStoredFieldsFormat#CompressingStoredFieldsFormat(String, String,
* CompressionMode, int, int, int)
* @see Lucene90CompressingStoredFieldsFormat#Lucene90CompressingStoredFieldsFormat(String,
* String, CompressionMode, int, int, int)
*/
public CompressingStoredFieldsFormat(
public Lucene90CompressingStoredFieldsFormat(
String formatName,
CompressionMode compressionMode,
int chunkSize,
@ -65,7 +66,7 @@ public class CompressingStoredFieldsFormat extends StoredFieldsFormat {
}
/**
* Create a new {@link CompressingStoredFieldsFormat}.
* Create a new {@link Lucene90CompressingStoredFieldsFormat}.
*
* <p><code>formatName</code> is the name of the format. This name will be used in the file
* formats to perform {@link CodecUtil#checkIndexHeader codec header checks}.
@ -76,7 +77,8 @@ public class CompressingStoredFieldsFormat extends StoredFieldsFormat {
* <p>The <code>compressionMode</code> parameter allows you to choose between compression
* algorithms that have various compression and decompression speeds so that you can pick the one
* that best fits your indexing and searching throughput. You should never instantiate two {@link
* CompressingStoredFieldsFormat}s that have the same name but different {@link CompressionMode}s.
* Lucene90CompressingStoredFieldsFormat}s that have the same name but different {@link
* CompressionMode}s.
*
* <p><code>chunkSize</code> is the minimum byte size of a chunk of documents. A value of <code>1
* </code> can make sense if there is redundancy across fields. <code>maxDocsPerChunk</code> is an
@ -94,7 +96,7 @@ public class CompressingStoredFieldsFormat extends StoredFieldsFormat {
* @param blockShift the log in base 2 of number of chunks to store in an index block
* @see CompressionMode
*/
public CompressingStoredFieldsFormat(
public Lucene90CompressingStoredFieldsFormat(
String formatName,
String segmentSuffix,
CompressionMode compressionMode,
@ -128,14 +130,14 @@ public class CompressingStoredFieldsFormat extends StoredFieldsFormat {
@Override
public StoredFieldsReader fieldsReader(
Directory directory, SegmentInfo si, FieldInfos fn, IOContext context) throws IOException {
return new CompressingStoredFieldsReader(
return new Lucene90CompressingStoredFieldsReader(
directory, si, segmentSuffix, fn, context, formatName, compressionMode);
}
@Override
public StoredFieldsWriter fieldsWriter(Directory directory, SegmentInfo si, IOContext context)
throws IOException {
return new CompressingStoredFieldsWriter(
return new Lucene90CompressingStoredFieldsWriter(
directory,
si,
segmentSuffix,

View File

@ -0,0 +1,756 @@
/*
* 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.compressing;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsWriter.BYTE_ARR;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsWriter.DAY;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsWriter.DAY_ENCODING;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsWriter.FIELDS_EXTENSION;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsWriter.HOUR;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsWriter.HOUR_ENCODING;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsWriter.INDEX_CODEC_NAME;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsWriter.INDEX_EXTENSION;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsWriter.META_EXTENSION;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsWriter.META_VERSION_START;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsWriter.NUMERIC_DOUBLE;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsWriter.NUMERIC_FLOAT;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsWriter.NUMERIC_INT;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsWriter.NUMERIC_LONG;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsWriter.SECOND;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsWriter.SECOND_ENCODING;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsWriter.STRING;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsWriter.TYPE_BITS;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsWriter.TYPE_MASK;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsWriter.VERSION_CURRENT;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsWriter.VERSION_START;
import java.io.EOFException;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.compressing.CompressionMode;
import org.apache.lucene.codecs.compressing.Decompressor;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.StoredFieldVisitor;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BitUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LongsRef;
import org.apache.lucene.util.packed.PackedInts;
/**
* {@link StoredFieldsReader} impl for {@link Lucene90CompressingStoredFieldsFormat}.
*
* @lucene.experimental
*/
public final class Lucene90CompressingStoredFieldsReader extends StoredFieldsReader {
private final int version;
private final FieldInfos fieldInfos;
private final FieldsIndex indexReader;
private final long maxPointer;
private final IndexInput fieldsStream;
private final int chunkSize;
private final int packedIntsVersion;
private final CompressionMode compressionMode;
private final Decompressor decompressor;
private final int numDocs;
private final boolean merging;
private final BlockState state;
private final long numDirtyChunks; // number of incomplete compressed blocks written
private final long numDirtyDocs; // cumulative number of missing docs in incomplete chunks
private boolean closed;
// used by clone
private Lucene90CompressingStoredFieldsReader(
Lucene90CompressingStoredFieldsReader reader, boolean merging) {
this.version = reader.version;
this.fieldInfos = reader.fieldInfos;
this.fieldsStream = reader.fieldsStream.clone();
this.indexReader = reader.indexReader.clone();
this.maxPointer = reader.maxPointer;
this.chunkSize = reader.chunkSize;
this.packedIntsVersion = reader.packedIntsVersion;
this.compressionMode = reader.compressionMode;
this.decompressor = reader.decompressor.clone();
this.numDocs = reader.numDocs;
this.numDirtyChunks = reader.numDirtyChunks;
this.numDirtyDocs = reader.numDirtyDocs;
this.merging = merging;
this.state = new BlockState();
this.closed = false;
}
/** Sole constructor. */
public Lucene90CompressingStoredFieldsReader(
Directory d,
SegmentInfo si,
String segmentSuffix,
FieldInfos fn,
IOContext context,
String formatName,
CompressionMode compressionMode)
throws IOException {
this.compressionMode = compressionMode;
final String segment = si.name;
boolean success = false;
fieldInfos = fn;
numDocs = si.maxDoc();
final String fieldsStreamFN =
IndexFileNames.segmentFileName(segment, segmentSuffix, FIELDS_EXTENSION);
ChecksumIndexInput metaIn = null;
try {
// Open the data file
fieldsStream = d.openInput(fieldsStreamFN, context);
version =
CodecUtil.checkIndexHeader(
fieldsStream, formatName, VERSION_START, VERSION_CURRENT, si.getId(), segmentSuffix);
assert CodecUtil.indexHeaderLength(formatName, segmentSuffix)
== fieldsStream.getFilePointer();
final String metaStreamFN =
IndexFileNames.segmentFileName(segment, segmentSuffix, META_EXTENSION);
metaIn = d.openChecksumInput(metaStreamFN, IOContext.READONCE);
CodecUtil.checkIndexHeader(
metaIn,
INDEX_CODEC_NAME + "Meta",
META_VERSION_START,
version,
si.getId(),
segmentSuffix);
chunkSize = metaIn.readVInt();
packedIntsVersion = metaIn.readVInt();
decompressor = compressionMode.newDecompressor();
this.merging = false;
this.state = new BlockState();
// NOTE: data file is too costly to verify checksum against all the bytes on open,
// but for now we at least verify proper structure of the checksum footer: which looks
// for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption
// such as file truncation.
CodecUtil.retrieveChecksum(fieldsStream);
long maxPointer = -1;
FieldsIndex indexReader = null;
FieldsIndexReader fieldsIndexReader =
new FieldsIndexReader(
d, si.name, segmentSuffix, INDEX_EXTENSION, INDEX_CODEC_NAME, si.getId(), metaIn);
indexReader = fieldsIndexReader;
maxPointer = fieldsIndexReader.getMaxPointer();
this.maxPointer = maxPointer;
this.indexReader = indexReader;
numDirtyChunks = metaIn.readVLong();
numDirtyDocs = metaIn.readVLong();
if (metaIn != null) {
CodecUtil.checkFooter(metaIn, null);
metaIn.close();
}
success = true;
} catch (Throwable t) {
if (metaIn != null) {
CodecUtil.checkFooter(metaIn, t);
throw new AssertionError("unreachable");
} else {
throw t;
}
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(this, metaIn);
}
}
}
/** @throws AlreadyClosedException if this FieldsReader is closed */
private void ensureOpen() throws AlreadyClosedException {
if (closed) {
throw new AlreadyClosedException("this FieldsReader is closed");
}
}
/** Close the underlying {@link IndexInput}s. */
@Override
public void close() throws IOException {
if (!closed) {
IOUtils.close(indexReader, fieldsStream);
closed = true;
}
}
private static void readField(DataInput in, StoredFieldVisitor visitor, FieldInfo info, int bits)
throws IOException {
switch (bits & TYPE_MASK) {
case BYTE_ARR:
int length = in.readVInt();
byte[] data = new byte[length];
in.readBytes(data, 0, length);
visitor.binaryField(info, data);
break;
case STRING:
visitor.stringField(info, in.readString());
break;
case NUMERIC_INT:
visitor.intField(info, in.readZInt());
break;
case NUMERIC_FLOAT:
visitor.floatField(info, readZFloat(in));
break;
case NUMERIC_LONG:
visitor.longField(info, readTLong(in));
break;
case NUMERIC_DOUBLE:
visitor.doubleField(info, readZDouble(in));
break;
default:
throw new AssertionError("Unknown type flag: " + Integer.toHexString(bits));
}
}
private static void skipField(DataInput in, int bits) throws IOException {
switch (bits & TYPE_MASK) {
case BYTE_ARR:
case STRING:
final int length = in.readVInt();
in.skipBytes(length);
break;
case NUMERIC_INT:
in.readZInt();
break;
case NUMERIC_FLOAT:
readZFloat(in);
break;
case NUMERIC_LONG:
readTLong(in);
break;
case NUMERIC_DOUBLE:
readZDouble(in);
break;
default:
throw new AssertionError("Unknown type flag: " + Integer.toHexString(bits));
}
}
/**
* Reads a float in a variable-length format. Reads between one and five bytes. Small integral
* values typically take fewer bytes.
*/
static float readZFloat(DataInput in) throws IOException {
int b = in.readByte() & 0xFF;
if (b == 0xFF) {
// negative value
return Float.intBitsToFloat(in.readInt());
} else if ((b & 0x80) != 0) {
// small integer [-1..125]
return (b & 0x7f) - 1;
} else {
// positive float
int bits = b << 24 | ((in.readShort() & 0xFFFF) << 8) | (in.readByte() & 0xFF);
return Float.intBitsToFloat(bits);
}
}
/**
* Reads a double in a variable-length format. Reads between one and nine bytes. Small integral
* values typically take fewer bytes.
*/
static double readZDouble(DataInput in) throws IOException {
int b = in.readByte() & 0xFF;
if (b == 0xFF) {
// negative value
return Double.longBitsToDouble(in.readLong());
} else if (b == 0xFE) {
// float
return Float.intBitsToFloat(in.readInt());
} else if ((b & 0x80) != 0) {
// small integer [-1..124]
return (b & 0x7f) - 1;
} else {
// positive double
long bits =
((long) b) << 56
| ((in.readInt() & 0xFFFFFFFFL) << 24)
| ((in.readShort() & 0xFFFFL) << 8)
| (in.readByte() & 0xFFL);
return Double.longBitsToDouble(bits);
}
}
/**
* Reads a long in a variable-length format. Reads between one andCorePropLo nine bytes. Small
* values typically take fewer bytes.
*/
static long readTLong(DataInput in) throws IOException {
int header = in.readByte() & 0xFF;
long bits = header & 0x1F;
if ((header & 0x20) != 0) {
// continuation bit
bits |= in.readVLong() << 5;
}
long l = BitUtil.zigZagDecode(bits);
switch (header & DAY_ENCODING) {
case SECOND_ENCODING:
l *= SECOND;
break;
case HOUR_ENCODING:
l *= HOUR;
break;
case DAY_ENCODING:
l *= DAY;
break;
case 0:
// uncompressed
break;
default:
throw new AssertionError();
}
return l;
}
/**
* A serialized document, you need to decode its input in order to get an actual {@link Document}.
*/
static class SerializedDocument {
// the serialized data
final DataInput in;
// the number of bytes on which the document is encoded
final int length;
// the number of stored fields
final int numStoredFields;
private SerializedDocument(DataInput in, int length, int numStoredFields) {
this.in = in;
this.length = length;
this.numStoredFields = numStoredFields;
}
}
/** Keeps state about the current block of documents. */
private class BlockState {
private int docBase, chunkDocs;
// whether the block has been sliced, this happens for large documents
private boolean sliced;
private long[] offsets = LongsRef.EMPTY_LONGS;
private long[] numStoredFields = LongsRef.EMPTY_LONGS;
// the start pointer at which you can read the compressed documents
private long startPointer;
private final BytesRef spare;
private final BytesRef bytes;
BlockState() {
if (merging) {
spare = new BytesRef();
bytes = new BytesRef();
} else {
spare = bytes = null;
}
}
boolean contains(int docID) {
return docID >= docBase && docID < docBase + chunkDocs;
}
/** Reset this block so that it stores state for the block that contains the given doc id. */
void reset(int docID) throws IOException {
boolean success = false;
try {
doReset(docID);
success = true;
} finally {
if (success == false) {
// if the read failed, set chunkDocs to 0 so that it does not
// contain any docs anymore and is not reused. This should help
// get consistent exceptions when trying to get several
// documents which are in the same corrupted block since it will
// force the header to be decoded again
chunkDocs = 0;
}
}
}
private void doReset(int docID) throws IOException {
docBase = fieldsStream.readVInt();
final int token = fieldsStream.readVInt();
chunkDocs = token >>> 1;
if (contains(docID) == false || docBase + chunkDocs > numDocs) {
throw new CorruptIndexException(
"Corrupted: docID="
+ docID
+ ", docBase="
+ docBase
+ ", chunkDocs="
+ chunkDocs
+ ", numDocs="
+ numDocs,
fieldsStream);
}
sliced = (token & 1) != 0;
offsets = ArrayUtil.grow(offsets, chunkDocs + 1);
numStoredFields = ArrayUtil.grow(numStoredFields, chunkDocs);
if (chunkDocs == 1) {
numStoredFields[0] = fieldsStream.readVInt();
offsets[1] = fieldsStream.readVInt();
} else {
// Number of stored fields per document
final int bitsPerStoredFields = fieldsStream.readVInt();
if (bitsPerStoredFields == 0) {
Arrays.fill(numStoredFields, 0, chunkDocs, fieldsStream.readVInt());
} else if (bitsPerStoredFields > 31) {
throw new CorruptIndexException(
"bitsPerStoredFields=" + bitsPerStoredFields, fieldsStream);
} else {
final PackedInts.ReaderIterator it =
PackedInts.getReaderIteratorNoHeader(
fieldsStream,
PackedInts.Format.PACKED,
packedIntsVersion,
chunkDocs,
bitsPerStoredFields,
1024);
for (int i = 0; i < chunkDocs; ) {
final LongsRef next = it.next(Integer.MAX_VALUE);
System.arraycopy(next.longs, next.offset, numStoredFields, i, next.length);
i += next.length;
}
}
// The stream encodes the length of each document and we decode
// it into a list of monotonically increasing offsets
final int bitsPerLength = fieldsStream.readVInt();
if (bitsPerLength == 0) {
final int length = fieldsStream.readVInt();
for (int i = 0; i < chunkDocs; ++i) {
offsets[1 + i] = (1 + i) * length;
}
} else if (bitsPerStoredFields > 31) {
throw new CorruptIndexException("bitsPerLength=" + bitsPerLength, fieldsStream);
} else {
final PackedInts.ReaderIterator it =
PackedInts.getReaderIteratorNoHeader(
fieldsStream,
PackedInts.Format.PACKED,
packedIntsVersion,
chunkDocs,
bitsPerLength,
1024);
for (int i = 0; i < chunkDocs; ) {
final LongsRef next = it.next(Integer.MAX_VALUE);
System.arraycopy(next.longs, next.offset, offsets, i + 1, next.length);
i += next.length;
}
for (int i = 0; i < chunkDocs; ++i) {
offsets[i + 1] += offsets[i];
}
}
// Additional validation: only the empty document has a serialized length of 0
for (int i = 0; i < chunkDocs; ++i) {
final long len = offsets[i + 1] - offsets[i];
final long storedFields = numStoredFields[i];
if ((len == 0) != (storedFields == 0)) {
throw new CorruptIndexException(
"length=" + len + ", numStoredFields=" + storedFields, fieldsStream);
}
}
}
startPointer = fieldsStream.getFilePointer();
if (merging) {
final int totalLength = Math.toIntExact(offsets[chunkDocs]);
// decompress eagerly
if (sliced) {
bytes.offset = bytes.length = 0;
for (int decompressed = 0; decompressed < totalLength; ) {
final int toDecompress = Math.min(totalLength - decompressed, chunkSize);
decompressor.decompress(fieldsStream, toDecompress, 0, toDecompress, spare);
bytes.bytes = ArrayUtil.grow(bytes.bytes, bytes.length + spare.length);
System.arraycopy(spare.bytes, spare.offset, bytes.bytes, bytes.length, spare.length);
bytes.length += spare.length;
decompressed += toDecompress;
}
} else {
decompressor.decompress(fieldsStream, totalLength, 0, totalLength, bytes);
}
if (bytes.length != totalLength) {
throw new CorruptIndexException(
"Corrupted: expected chunk size = " + totalLength + ", got " + bytes.length,
fieldsStream);
}
}
}
/**
* Get the serialized representation of the given docID. This docID has to be contained in the
* current block.
*/
SerializedDocument document(int docID) throws IOException {
if (contains(docID) == false) {
throw new IllegalArgumentException();
}
final int index = docID - docBase;
final int offset = Math.toIntExact(offsets[index]);
final int length = Math.toIntExact(offsets[index + 1]) - offset;
final int totalLength = Math.toIntExact(offsets[chunkDocs]);
final int numStoredFields = Math.toIntExact(this.numStoredFields[index]);
final BytesRef bytes;
if (merging) {
bytes = this.bytes;
} else {
bytes = new BytesRef();
}
final DataInput documentInput;
if (length == 0) {
// empty
documentInput = new ByteArrayDataInput();
} else if (merging) {
// already decompressed
documentInput = new ByteArrayDataInput(bytes.bytes, bytes.offset + offset, length);
} else if (sliced) {
fieldsStream.seek(startPointer);
decompressor.decompress(
fieldsStream, chunkSize, offset, Math.min(length, chunkSize - offset), bytes);
documentInput =
new DataInput() {
int decompressed = bytes.length;
void fillBuffer() throws IOException {
assert decompressed <= length;
if (decompressed == length) {
throw new EOFException();
}
final int toDecompress = Math.min(length - decompressed, chunkSize);
decompressor.decompress(fieldsStream, toDecompress, 0, toDecompress, bytes);
decompressed += toDecompress;
}
@Override
public byte readByte() throws IOException {
if (bytes.length == 0) {
fillBuffer();
}
--bytes.length;
return bytes.bytes[bytes.offset++];
}
@Override
public void readBytes(byte[] b, int offset, int len) throws IOException {
while (len > bytes.length) {
System.arraycopy(bytes.bytes, bytes.offset, b, offset, bytes.length);
len -= bytes.length;
offset += bytes.length;
fillBuffer();
}
System.arraycopy(bytes.bytes, bytes.offset, b, offset, len);
bytes.offset += len;
bytes.length -= len;
}
@Override
public void skipBytes(long numBytes) throws IOException {
if (numBytes < 0) {
throw new IllegalArgumentException("numBytes must be >= 0, got " + numBytes);
}
while (numBytes > bytes.length) {
numBytes -= bytes.length;
fillBuffer();
}
bytes.offset += numBytes;
bytes.length -= numBytes;
}
};
} else {
fieldsStream.seek(startPointer);
decompressor.decompress(fieldsStream, totalLength, offset, length, bytes);
assert bytes.length == length;
documentInput = new ByteArrayDataInput(bytes.bytes, bytes.offset, bytes.length);
}
return new SerializedDocument(documentInput, length, numStoredFields);
}
}
SerializedDocument document(int docID) throws IOException {
if (state.contains(docID) == false) {
fieldsStream.seek(indexReader.getStartPointer(docID));
state.reset(docID);
}
assert state.contains(docID);
return state.document(docID);
}
@Override
public void visitDocument(int docID, StoredFieldVisitor visitor) throws IOException {
final SerializedDocument doc = document(docID);
for (int fieldIDX = 0; fieldIDX < doc.numStoredFields; fieldIDX++) {
final long infoAndBits = doc.in.readVLong();
final int fieldNumber = (int) (infoAndBits >>> TYPE_BITS);
final FieldInfo fieldInfo = fieldInfos.fieldInfo(fieldNumber);
final int bits = (int) (infoAndBits & TYPE_MASK);
assert bits <= NUMERIC_DOUBLE : "bits=" + Integer.toHexString(bits);
switch (visitor.needsField(fieldInfo)) {
case YES:
readField(doc.in, visitor, fieldInfo, bits);
break;
case NO:
if (fieldIDX
== doc.numStoredFields - 1) { // don't skipField on last field value; treat like STOP
return;
}
skipField(doc.in, bits);
break;
case STOP:
return;
}
}
}
@Override
public StoredFieldsReader clone() {
ensureOpen();
return new Lucene90CompressingStoredFieldsReader(this, false);
}
@Override
public StoredFieldsReader getMergeInstance() {
ensureOpen();
return new Lucene90CompressingStoredFieldsReader(this, true);
}
int getVersion() {
return version;
}
CompressionMode getCompressionMode() {
return compressionMode;
}
FieldsIndex getIndexReader() {
return indexReader;
}
long getMaxPointer() {
return maxPointer;
}
IndexInput getFieldsStream() {
return fieldsStream;
}
int getChunkSize() {
return chunkSize;
}
long getNumDirtyDocs() {
if (version != VERSION_CURRENT) {
throw new IllegalStateException(
"getNumDirtyDocs should only ever get called when the reader is on the current version");
}
assert numDirtyDocs >= 0;
return numDirtyDocs;
}
long getNumDirtyChunks() {
if (version != VERSION_CURRENT) {
throw new IllegalStateException(
"getNumDirtyChunks should only ever get called when the reader is on the current version");
}
assert numDirtyChunks >= 0;
return numDirtyChunks;
}
int getNumDocs() {
return numDocs;
}
int getPackedIntsVersion() {
return packedIntsVersion;
}
@Override
public long ramBytesUsed() {
return indexReader.ramBytesUsed();
}
@Override
public Collection<Accountable> getChildResources() {
return Collections.singleton(Accountables.namedAccountable("stored field index", indexReader));
}
@Override
public void checkIntegrity() throws IOException {
indexReader.checkIntegrity();
CodecUtil.checksumEntireFile(fieldsStream);
}
@Override
public String toString() {
return getClass().getSimpleName()
+ "(mode="
+ compressionMode
+ ",chunksize="
+ chunkSize
+ ")";
}
}

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.compressing;
package org.apache.lucene.codecs.lucene90.compressing;
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
@ -24,7 +24,10 @@ import java.util.List;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.StoredFieldsWriter;
import org.apache.lucene.codecs.compressing.CompressingStoredFieldsReader.SerializedDocument;
import org.apache.lucene.codecs.compressing.CompressionMode;
import org.apache.lucene.codecs.compressing.Compressor;
import org.apache.lucene.codecs.compressing.MatchingReaders;
import org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsReader.SerializedDocument;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.DocIDMerger;
import org.apache.lucene.index.FieldInfo;
@ -47,11 +50,11 @@ import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.packed.PackedInts;
/**
* {@link StoredFieldsWriter} impl for {@link CompressingStoredFieldsFormat}.
* {@link StoredFieldsWriter} impl for {@link Lucene90CompressingStoredFieldsFormat}.
*
* @lucene.experimental
*/
public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
public final class Lucene90CompressingStoredFieldsWriter extends StoredFieldsWriter {
/** Extension of stored fields file */
public static final String FIELDS_EXTENSION = "fdt";
@ -60,7 +63,7 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
/** Extension of stored fields meta */
public static final String META_EXTENSION = "fdm";
/** Codec name for the index. */
public static final String INDEX_CODEC_NAME = "Lucene85FieldsIndex";
public static final String INDEX_CODEC_NAME = "Lucene90FieldsIndex";
static final int STRING = 0x00;
static final int BYTE_ARR = 0x01;
@ -73,11 +76,7 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
static final int TYPE_MASK = (int) PackedInts.maxValue(TYPE_BITS);
static final int VERSION_START = 1;
static final int VERSION_OFFHEAP_INDEX = 2;
/** Version where all metadata were moved to the meta file. */
static final int VERSION_META = 3;
static final int VERSION_CURRENT = VERSION_META;
static final int VERSION_CURRENT = VERSION_START;
static final int META_VERSION_START = 0;
private final String segment;
@ -99,7 +98,7 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
private long numDirtyDocs; // cumulative number of missing docs in incomplete chunks
/** Sole constructor. */
CompressingStoredFieldsWriter(
Lucene90CompressingStoredFieldsWriter(
Directory directory,
SegmentInfo si,
String segmentSuffix,
@ -514,7 +513,7 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
// we try to be extra safe with this impl, but add an escape hatch to
// have a workaround for undiscovered bugs.
static final String BULK_MERGE_ENABLED_SYSPROP =
CompressingStoredFieldsWriter.class.getName() + ".enableBulkMerge";
Lucene90CompressingStoredFieldsWriter.class.getName() + ".enableBulkMerge";
static final boolean BULK_MERGE_ENABLED;
static {
@ -540,9 +539,9 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
List<CompressingStoredFieldsMergeSub> subs = new ArrayList<>();
for (int i = 0; i < mergeState.storedFieldsReaders.length; i++) {
if (matching.matchingReaders[i]
&& mergeState.storedFieldsReaders[i] instanceof CompressingStoredFieldsReader) {
CompressingStoredFieldsReader storedFieldsReader =
(CompressingStoredFieldsReader) mergeState.storedFieldsReaders[i];
&& mergeState.storedFieldsReaders[i] instanceof Lucene90CompressingStoredFieldsReader) {
Lucene90CompressingStoredFieldsReader storedFieldsReader =
(Lucene90CompressingStoredFieldsReader) mergeState.storedFieldsReaders[i];
storedFieldsReader.checkIntegrity();
subs.add(
new CompressingStoredFieldsMergeSub(
@ -572,12 +571,12 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
for (int readerIndex = 0; readerIndex < numReaders; readerIndex++) {
MergeVisitor visitor = new MergeVisitor(mergeState, readerIndex);
CompressingStoredFieldsReader matchingFieldsReader = null;
Lucene90CompressingStoredFieldsReader matchingFieldsReader = null;
if (matching.matchingReaders[readerIndex]) {
final StoredFieldsReader fieldsReader = mergeState.storedFieldsReaders[readerIndex];
// we can only bulk-copy if the matching reader is also a CompressingStoredFieldsReader
if (fieldsReader != null && fieldsReader instanceof CompressingStoredFieldsReader) {
matchingFieldsReader = (CompressingStoredFieldsReader) fieldsReader;
if (fieldsReader != null && fieldsReader instanceof Lucene90CompressingStoredFieldsReader) {
matchingFieldsReader = (Lucene90CompressingStoredFieldsReader) fieldsReader;
}
}
@ -709,19 +708,19 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
* some worst-case situations (e.g. frequent reopen with tiny flushes), over time the compression
* ratio can degrade. This is a safety switch.
*/
boolean tooDirty(CompressingStoredFieldsReader candidate) {
boolean tooDirty(Lucene90CompressingStoredFieldsReader candidate) {
// more than 1% dirty, or more than hard limit of 1024 dirty chunks
return candidate.getNumDirtyChunks() > 1024
|| candidate.getNumDirtyDocs() * 100 > candidate.getNumDocs();
}
private static class CompressingStoredFieldsMergeSub extends DocIDMerger.Sub {
private final CompressingStoredFieldsReader reader;
private final Lucene90CompressingStoredFieldsReader reader;
private final int maxDoc;
int docID = -1;
CompressingStoredFieldsMergeSub(
CompressingStoredFieldsReader reader, MergeState.DocMap docMap, int maxDoc) {
Lucene90CompressingStoredFieldsReader reader, MergeState.DocMap docMap, int maxDoc) {
super(docMap);
this.maxDoc = maxDoc;
this.reader = reader;

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.compressing;
package org.apache.lucene.codecs.lucene90.compressing;
import java.io.IOException;
import org.apache.lucene.codecs.CodecUtil;
@ -22,6 +22,7 @@ import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.codecs.TermVectorsWriter;
import org.apache.lucene.codecs.compressing.CompressionMode;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory;

View File

@ -14,20 +14,20 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.compressing;
package org.apache.lucene.codecs.lucene90.compressing;
import static org.apache.lucene.codecs.compressing.Lucene90CompressingTermVectorsWriter.FLAGS_BITS;
import static org.apache.lucene.codecs.compressing.Lucene90CompressingTermVectorsWriter.META_VERSION_START;
import static org.apache.lucene.codecs.compressing.Lucene90CompressingTermVectorsWriter.OFFSETS;
import static org.apache.lucene.codecs.compressing.Lucene90CompressingTermVectorsWriter.PACKED_BLOCK_SIZE;
import static org.apache.lucene.codecs.compressing.Lucene90CompressingTermVectorsWriter.PAYLOADS;
import static org.apache.lucene.codecs.compressing.Lucene90CompressingTermVectorsWriter.POSITIONS;
import static org.apache.lucene.codecs.compressing.Lucene90CompressingTermVectorsWriter.VECTORS_EXTENSION;
import static org.apache.lucene.codecs.compressing.Lucene90CompressingTermVectorsWriter.VECTORS_INDEX_CODEC_NAME;
import static org.apache.lucene.codecs.compressing.Lucene90CompressingTermVectorsWriter.VECTORS_INDEX_EXTENSION;
import static org.apache.lucene.codecs.compressing.Lucene90CompressingTermVectorsWriter.VECTORS_META_EXTENSION;
import static org.apache.lucene.codecs.compressing.Lucene90CompressingTermVectorsWriter.VERSION_CURRENT;
import static org.apache.lucene.codecs.compressing.Lucene90CompressingTermVectorsWriter.VERSION_START;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingTermVectorsWriter.FLAGS_BITS;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingTermVectorsWriter.META_VERSION_START;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingTermVectorsWriter.OFFSETS;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingTermVectorsWriter.PACKED_BLOCK_SIZE;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingTermVectorsWriter.PAYLOADS;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingTermVectorsWriter.POSITIONS;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingTermVectorsWriter.VECTORS_EXTENSION;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingTermVectorsWriter.VECTORS_INDEX_CODEC_NAME;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingTermVectorsWriter.VECTORS_INDEX_EXTENSION;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingTermVectorsWriter.VECTORS_META_EXTENSION;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingTermVectorsWriter.VERSION_CURRENT;
import static org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingTermVectorsWriter.VERSION_START;
import java.io.Closeable;
import java.io.IOException;
@ -37,6 +37,8 @@ import java.util.Iterator;
import java.util.NoSuchElementException;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.codecs.compressing.CompressionMode;
import org.apache.lucene.codecs.compressing.Decompressor;
import org.apache.lucene.index.BaseTermsEnum;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo;

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.compressing;
package org.apache.lucene.codecs.lucene90.compressing;
import java.io.IOException;
import java.util.ArrayDeque;
@ -28,6 +28,9 @@ import java.util.TreeSet;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.codecs.TermVectorsWriter;
import org.apache.lucene.codecs.compressing.CompressionMode;
import org.apache.lucene.codecs.compressing.Compressor;
import org.apache.lucene.codecs.compressing.MatchingReaders;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;

View File

@ -15,8 +15,5 @@
* limitations under the License.
*/
/**
* Components from the Lucene 8.7 index format. See {@link org.apache.lucene.codecs.lucene90} for an
* overview of the current index format.
*/
package org.apache.lucene.codecs.lucene87;
/** Lucene 9.0 compressing format. */
package org.apache.lucene.codecs.lucene90.compressing;

View File

@ -146,7 +146,7 @@
* information about how the segment is sorted
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90FieldInfosFormat Field names}. This
* contains metadata about the set of named fields used in the index.
* <li>{@link org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat Stored Field values}.
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Stored Field values}.
* This contains, for each document, a list of attribute-value pairs, where the attributes are
* field names. These are used to store auxiliary information about the document, such as its
* title, url, or an identifier to access a database. The set of stored fields are what is
@ -245,12 +245,12 @@
* <td>Stores information about the fields</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat Field Index}</td>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Field Index}</td>
* <td>.fdx</td>
* <td>Contains pointers to field data</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat Field Data}</td>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Field Data}</td>
* <td>.fdt</td>
* <td>The stored fields for documents</td>
* </tr>

View File

@ -26,10 +26,10 @@ import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.StoredFieldsWriter;
import org.apache.lucene.codecs.compressing.CompressingStoredFieldsFormat;
import org.apache.lucene.codecs.compressing.CompressionMode;
import org.apache.lucene.codecs.compressing.Compressor;
import org.apache.lucene.codecs.compressing.Decompressor;
import org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsFormat;
import org.apache.lucene.document.StoredField;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
@ -79,7 +79,8 @@ final class SortingStoredFieldsConsumer extends StoredFieldsConsumer {
}
};
private static final StoredFieldsFormat TEMP_STORED_FIELDS_FORMAT =
new CompressingStoredFieldsFormat("TempStoredFields", NO_COMPRESSION, 128 * 1024, 1, 10);
new Lucene90CompressingStoredFieldsFormat(
"TempStoredFields", NO_COMPRESSION, 128 * 1024, 1, 10);
TrackingTmpOutputDirectoryWrapper tmpDirectory;
SortingStoredFieldsConsumer(Codec codec, Directory directory, SegmentInfo info) {

View File

@ -24,7 +24,7 @@ import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.codecs.TermVectorsWriter;
import org.apache.lucene.codecs.compressing.Lucene90CompressingTermVectorsFormat;
import org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingTermVectorsFormat;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FlushInfo;

View File

@ -0,0 +1,28 @@
/*
* 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.index.BaseStoredFieldsFormatTestCase;
import org.apache.lucene.util.TestUtil;
public class TestLucene90StoredFieldsFormat extends BaseStoredFieldsFormatTestCase {
@Override
protected Codec getCodec() {
return TestUtil.getDefaultCodec();
}
}

View File

@ -0,0 +1,85 @@
/*
* 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 com.carrotsearch.randomizedtesting.generators.RandomPicks;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.lucene90.Lucene90Codec.Mode;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.StoredField;
import org.apache.lucene.index.BaseStoredFieldsFormatTestCase;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.store.Directory;
public class TestLucene90StoredFieldsFormatHighCompression extends BaseStoredFieldsFormatTestCase {
@Override
protected Codec getCodec() {
return new Lucene90Codec(Mode.BEST_COMPRESSION);
}
/**
* Change compression params (leaving it the same for old segments) and tests that nothing breaks.
*/
public void testMixedCompressions() throws Exception {
Directory dir = newDirectory();
for (int i = 0; i < 10; i++) {
IndexWriterConfig iwc = newIndexWriterConfig();
iwc.setCodec(new Lucene90Codec(RandomPicks.randomFrom(random(), Mode.values())));
IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig());
Document doc = new Document();
doc.add(new StoredField("field1", "value1"));
doc.add(new StoredField("field2", "value2"));
iw.addDocument(doc);
if (random().nextInt(4) == 0) {
iw.forceMerge(1);
}
iw.commit();
iw.close();
}
DirectoryReader ir = DirectoryReader.open(dir);
assertEquals(10, ir.numDocs());
for (int i = 0; i < 10; i++) {
Document doc = ir.document(i);
assertEquals("value1", doc.get("field1"));
assertEquals("value2", doc.get("field2"));
}
ir.close();
// checkindex
dir.close();
}
public void testInvalidOptions() {
expectThrows(
NullPointerException.class,
() -> {
new Lucene90Codec(null);
});
expectThrows(
NullPointerException.class,
() -> {
new Lucene90StoredFieldsFormat(null);
});
}
public void testShowJDKBugStatus() {
System.err.println("JDK is buggy (JDK-8252739): " + BugfixDeflater_JDK8252739.IS_BUGGY_JDK);
}
}

View File

@ -0,0 +1,26 @@
/*
* 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;
/** Test the merge instance of the Lucene90 stored fields format. */
public class TestLucene90StoredFieldsFormatMergeInstance extends TestLucene90StoredFieldsFormat {
@Override
protected boolean shouldTestMergeInstance() {
return true;
}
}

View File

@ -27,7 +27,7 @@ import java.util.HashSet;
import java.util.Set;
import java.util.function.Function;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter;
import org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsWriter;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexNotFoundException;
import org.apache.lucene.index.IndexReader;
@ -44,7 +44,7 @@ public class TestFileSwitchDirectory extends BaseDirectoryTestCase {
/** Test if writing doc stores to disk and everything else to ram works. */
public void testBasic() throws IOException {
Set<String> fileExtensions = new HashSet<>();
fileExtensions.add(CompressingStoredFieldsWriter.FIELDS_EXTENSION);
fileExtensions.add(Lucene90CompressingStoredFieldsWriter.FIELDS_EXTENSION);
fileExtensions.add("fdx");
fileExtensions.add("fdm");

View File

@ -22,12 +22,14 @@ import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.compressing.dummy.DummyCompressingCodec;
import org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsFormat;
import org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingTermVectorsFormat;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.packed.DirectMonotonicWriter;
/**
* A codec that uses {@link CompressingStoredFieldsFormat} for its stored fields and delegates to
* the default codec for everything else.
* A codec that uses {@link Lucene90CompressingStoredFieldsFormat} for its stored fields and
* delegates to the default codec for everything else.
*/
public abstract class CompressingCodec extends FilterCodec {
@ -101,7 +103,7 @@ public abstract class CompressingCodec extends FilterCodec {
RandomNumbers.randomIntBetween(random, 1, 1024));
}
private final CompressingStoredFieldsFormat storedFieldsFormat;
private final Lucene90CompressingStoredFieldsFormat storedFieldsFormat;
private final Lucene90CompressingTermVectorsFormat termVectorsFormat;
/** Creates a compressing codec with a given segment suffix */
@ -114,7 +116,7 @@ public abstract class CompressingCodec extends FilterCodec {
int blockShift) {
super(name, TestUtil.getDefaultCodec());
this.storedFieldsFormat =
new CompressingStoredFieldsFormat(
new Lucene90CompressingStoredFieldsFormat(
name, segmentSuffix, compressionMode, chunkSize, maxDocsPerChunk, blockShift);
this.termVectorsFormat =
new Lucene90CompressingTermVectorsFormat(

View File

@ -16,7 +16,7 @@
*/
package org.apache.lucene.codecs.compressing;
import org.apache.lucene.codecs.lucene87.DeflateWithPresetDictCompressionMode;
import org.apache.lucene.codecs.lucene90.DeflateWithPresetDictCompressionMode;
/** CompressionCodec that uses {@link DeflateWithPresetDictCompressionMode}. */
public class DeflateWithPresetCompressingCodec extends CompressingCodec {

View File

@ -16,7 +16,7 @@
*/
package org.apache.lucene.codecs.compressing;
import org.apache.lucene.codecs.lucene87.LZ4WithPresetDictCompressionMode;
import org.apache.lucene.codecs.lucene90.LZ4WithPresetDictCompressionMode;
/** CompressionCodec that uses {@link LZ4WithPresetDictCompressionMode}. */
public class LZ4WithPresetCompressingCodec extends CompressingCodec {

View File

@ -21,6 +21,6 @@
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
</head>
<body>
Support for testing {@link org.apache.lucene.codecs.compressing.CompressingStoredFieldsFormat}.
Support for testing {@link org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsFormat}.
</body>
</html>

View File

@ -14,13 +14,14 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.compressing;
package org.apache.lucene.codecs.lucene90.compressing;
import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
import java.io.IOException;
import java.util.Random;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.compressing.CompressingCodec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
@ -105,9 +106,9 @@ public class TestCompressingStoredFieldsFormat extends BaseStoredFieldsFormatTes
// round-trip small integer values
for (int i = Short.MIN_VALUE; i < Short.MAX_VALUE; i++) {
float f = (float) i;
CompressingStoredFieldsWriter.writeZFloat(out, f);
Lucene90CompressingStoredFieldsWriter.writeZFloat(out, f);
in.reset(buffer, 0, out.getPosition());
float g = CompressingStoredFieldsReader.readZFloat(in);
float g = Lucene90CompressingStoredFieldsReader.readZFloat(in);
assertTrue(in.eof());
assertEquals(Float.floatToIntBits(f), Float.floatToIntBits(g));
@ -130,9 +131,9 @@ public class TestCompressingStoredFieldsFormat extends BaseStoredFieldsFormatTes
};
for (float f : special) {
CompressingStoredFieldsWriter.writeZFloat(out, f);
Lucene90CompressingStoredFieldsWriter.writeZFloat(out, f);
in.reset(buffer, 0, out.getPosition());
float g = CompressingStoredFieldsReader.readZFloat(in);
float g = Lucene90CompressingStoredFieldsReader.readZFloat(in);
assertTrue(in.eof());
assertEquals(Float.floatToIntBits(f), Float.floatToIntBits(g));
out.reset(buffer);
@ -142,12 +143,12 @@ public class TestCompressingStoredFieldsFormat extends BaseStoredFieldsFormatTes
Random r = random();
for (int i = 0; i < 100000; i++) {
float f = r.nextFloat() * (random().nextInt(100) - 50);
CompressingStoredFieldsWriter.writeZFloat(out, f);
Lucene90CompressingStoredFieldsWriter.writeZFloat(out, f);
assertTrue(
"length=" + out.getPosition() + ", f=" + f,
out.getPosition() <= ((Float.floatToIntBits(f) >>> 31) == 1 ? 5 : 4));
in.reset(buffer, 0, out.getPosition());
float g = CompressingStoredFieldsReader.readZFloat(in);
float g = Lucene90CompressingStoredFieldsReader.readZFloat(in);
assertTrue(in.eof());
assertEquals(Float.floatToIntBits(f), Float.floatToIntBits(g));
out.reset(buffer);
@ -162,9 +163,9 @@ public class TestCompressingStoredFieldsFormat extends BaseStoredFieldsFormatTes
// round-trip small integer values
for (int i = Short.MIN_VALUE; i < Short.MAX_VALUE; i++) {
double x = (double) i;
CompressingStoredFieldsWriter.writeZDouble(out, x);
Lucene90CompressingStoredFieldsWriter.writeZDouble(out, x);
in.reset(buffer, 0, out.getPosition());
double y = CompressingStoredFieldsReader.readZDouble(in);
double y = Lucene90CompressingStoredFieldsReader.readZDouble(in);
assertTrue(in.eof());
assertEquals(Double.doubleToLongBits(x), Double.doubleToLongBits(y));
@ -187,9 +188,9 @@ public class TestCompressingStoredFieldsFormat extends BaseStoredFieldsFormatTes
};
for (double x : special) {
CompressingStoredFieldsWriter.writeZDouble(out, x);
Lucene90CompressingStoredFieldsWriter.writeZDouble(out, x);
in.reset(buffer, 0, out.getPosition());
double y = CompressingStoredFieldsReader.readZDouble(in);
double y = Lucene90CompressingStoredFieldsReader.readZDouble(in);
assertTrue(in.eof());
assertEquals(Double.doubleToLongBits(x), Double.doubleToLongBits(y));
out.reset(buffer);
@ -199,10 +200,10 @@ public class TestCompressingStoredFieldsFormat extends BaseStoredFieldsFormatTes
Random r = random();
for (int i = 0; i < 100000; i++) {
double x = r.nextDouble() * (random().nextInt(100) - 50);
CompressingStoredFieldsWriter.writeZDouble(out, x);
Lucene90CompressingStoredFieldsWriter.writeZDouble(out, x);
assertTrue("length=" + out.getPosition() + ", d=" + x, out.getPosition() <= (x < 0 ? 9 : 8));
in.reset(buffer, 0, out.getPosition());
double y = CompressingStoredFieldsReader.readZDouble(in);
double y = Lucene90CompressingStoredFieldsReader.readZDouble(in);
assertTrue(in.eof());
assertEquals(Double.doubleToLongBits(x), Double.doubleToLongBits(y));
out.reset(buffer);
@ -211,10 +212,10 @@ public class TestCompressingStoredFieldsFormat extends BaseStoredFieldsFormatTes
// same with floats
for (int i = 0; i < 100000; i++) {
double x = (double) (r.nextFloat() * (random().nextInt(100) - 50));
CompressingStoredFieldsWriter.writeZDouble(out, x);
Lucene90CompressingStoredFieldsWriter.writeZDouble(out, x);
assertTrue("length=" + out.getPosition() + ", d=" + x, out.getPosition() <= 5);
in.reset(buffer, 0, out.getPosition());
double y = CompressingStoredFieldsReader.readZDouble(in);
double y = Lucene90CompressingStoredFieldsReader.readZDouble(in);
assertTrue(in.eof());
assertEquals(Double.doubleToLongBits(x), Double.doubleToLongBits(y));
out.reset(buffer);
@ -230,9 +231,9 @@ public class TestCompressingStoredFieldsFormat extends BaseStoredFieldsFormatTes
for (int i = Short.MIN_VALUE; i < Short.MAX_VALUE; i++) {
for (long mul : new long[] {SECOND, HOUR, DAY}) {
long l1 = (long) i * mul;
CompressingStoredFieldsWriter.writeTLong(out, l1);
Lucene90CompressingStoredFieldsWriter.writeTLong(out, l1);
in.reset(buffer, 0, out.getPosition());
long l2 = CompressingStoredFieldsReader.readTLong(in);
long l2 = Lucene90CompressingStoredFieldsReader.readTLong(in);
assertTrue(in.eof());
assertEquals(l1, l2);
@ -262,9 +263,9 @@ public class TestCompressingStoredFieldsFormat extends BaseStoredFieldsFormatTes
default:
break;
}
CompressingStoredFieldsWriter.writeTLong(out, l1);
Lucene90CompressingStoredFieldsWriter.writeTLong(out, l1);
in.reset(buffer, 0, out.getPosition());
long l2 = CompressingStoredFieldsReader.readTLong(in);
long l2 = Lucene90CompressingStoredFieldsReader.readTLong(in);
assertTrue(in.eof());
assertEquals(l1, l2);
out.reset(buffer);
@ -297,7 +298,8 @@ public class TestCompressingStoredFieldsFormat extends BaseStoredFieldsFormatTes
// examine dirty counts:
for (LeafReaderContext leaf : ir2.leaves()) {
CodecReader sr = (CodecReader) leaf.reader();
CompressingStoredFieldsReader reader = (CompressingStoredFieldsReader) sr.getFieldsReader();
Lucene90CompressingStoredFieldsReader reader =
(Lucene90CompressingStoredFieldsReader) sr.getFieldsReader();
assertTrue(reader.getNumDirtyDocs() > 0);
assertTrue(reader.getNumDirtyDocs() < 100); // can't be gte the number of docs per chunk
assertEquals(1, reader.getNumDirtyChunks());
@ -310,7 +312,8 @@ public class TestCompressingStoredFieldsFormat extends BaseStoredFieldsFormatTes
ir.close();
ir = ir2;
CodecReader sr = (CodecReader) getOnlyLeafReader(ir);
CompressingStoredFieldsReader reader = (CompressingStoredFieldsReader) sr.getFieldsReader();
Lucene90CompressingStoredFieldsReader reader =
(Lucene90CompressingStoredFieldsReader) sr.getFieldsReader();
// we could get lucky, and have zero, but typically one.
assertTrue(reader.getNumDirtyChunks() <= 1);
ir.close();

View File

@ -14,11 +14,12 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.compressing;
package org.apache.lucene.codecs.lucene90.compressing;
import java.io.IOException;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.compressing.CompressingCodec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;

View File

@ -20,8 +20,8 @@ import java.io.IOException;
import java.util.Map;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat;
import org.apache.lucene.codecs.lucene90.Lucene90Codec.Mode;
import org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
import org.apache.lucene.index.SegmentInfo;
@ -117,11 +117,11 @@ public class TestCodecSupport extends SolrTestCaseJ4 {
SegmentInfos infos = SegmentInfos.readLatestCommit(searcher.getIndexReader().directory());
SegmentInfo info = infos.info(infos.size() - 1).info;
assertEquals("Expecting compression mode string to be " + expectedModeString +
" but got: " + info.getAttribute(Lucene87StoredFieldsFormat.MODE_KEY) +
" but got: " + info.getAttribute(Lucene90StoredFieldsFormat.MODE_KEY) +
"\n SegmentInfo: " + info +
"\n SegmentInfos: " + infos +
"\n Codec: " + core.getCodec(),
expectedModeString, info.getAttribute(Lucene87StoredFieldsFormat.MODE_KEY));
expectedModeString, info.getAttribute(Lucene90StoredFieldsFormat.MODE_KEY));
return null;
});
}