- * {@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Stored Field values}.
+ * 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
@@ -250,12 +250,12 @@
* Stores information about the fields |
*
*
- * {@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Field Index} |
+ * Field Index |
* .fdx |
* Contains pointers to field data |
*
*
- * {@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Field Data} |
+ * Field Data |
* .fdt |
* The stored fields for documents |
*
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene87/Lucene87Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene87/Lucene87Codec.java
new file mode 100644
index 00000000000..5ff407384e2
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene87/Lucene87Codec.java
@@ -0,0 +1,179 @@
+/*
+ * 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.lucene87;
+
+import java.util.Objects;
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.CompoundFormat;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.FieldInfosFormat;
+import org.apache.lucene.codecs.FilterCodec;
+import org.apache.lucene.codecs.LiveDocsFormat;
+import org.apache.lucene.codecs.NormsFormat;
+import org.apache.lucene.codecs.PointsFormat;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.SegmentInfoFormat;
+import org.apache.lucene.codecs.StoredFieldsFormat;
+import org.apache.lucene.codecs.TermVectorsFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50CompoundFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
+import org.apache.lucene.codecs.lucene60.Lucene60FieldInfosFormat;
+import org.apache.lucene.codecs.lucene80.Lucene80NormsFormat;
+import org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat;
+import org.apache.lucene.codecs.lucene86.Lucene86PointsFormat;
+import org.apache.lucene.codecs.lucene86.Lucene86SegmentInfoFormat;
+import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
+import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
+
+/**
+ * Implements the Lucene 8.6 index format, with configurable per-field postings
+ * and docvalues formats.
+ *
+ * If you want to reuse functionality of this codec in another codec, extend
+ * {@link FilterCodec}.
+ *
+ * @see org.apache.lucene.codecs.lucene86 package documentation for file format details.
+ *
+ * @lucene.experimental
+ */
+public class Lucene87Codec extends Codec {
+ private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat();
+ private final FieldInfosFormat fieldInfosFormat = new Lucene60FieldInfosFormat();
+ private final SegmentInfoFormat segmentInfosFormat = new Lucene86SegmentInfoFormat();
+ private final LiveDocsFormat liveDocsFormat = new Lucene50LiveDocsFormat();
+ private final CompoundFormat compoundFormat = new Lucene50CompoundFormat();
+ private final PointsFormat pointsFormat = new Lucene86PointsFormat();
+ private final PostingsFormat defaultFormat;
+
+ private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
+ @Override
+ public PostingsFormat getPostingsFormatForField(String field) {
+ return Lucene87Codec.this.getPostingsFormatForField(field);
+ }
+ };
+
+ private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
+ @Override
+ public DocValuesFormat getDocValuesFormatForField(String field) {
+ return Lucene87Codec.this.getDocValuesFormatForField(field);
+ }
+ };
+
+ private final StoredFieldsFormat storedFieldsFormat;
+
+ /**
+ * Instantiates a new codec.
+ */
+ public Lucene87Codec() {
+ this(Lucene87StoredFieldsFormat.Mode.BEST_SPEED);
+ }
+
+ /**
+ * Instantiates a new codec, specifying the stored fields compression
+ * mode to use.
+ * @param mode stored fields compression mode to use for newly
+ * flushed/merged segments.
+ */
+ public Lucene87Codec(Lucene87StoredFieldsFormat.Mode mode) {
+ super("Lucene87");
+ this.storedFieldsFormat = new Lucene87StoredFieldsFormat(Objects.requireNonNull(mode));
+ this.defaultFormat = new Lucene84PostingsFormat();
+ }
+
+ @Override
+ public final StoredFieldsFormat storedFieldsFormat() {
+ return storedFieldsFormat;
+ }
+
+ @Override
+ public final TermVectorsFormat termVectorsFormat() {
+ return vectorsFormat;
+ }
+
+ @Override
+ public final PostingsFormat postingsFormat() {
+ return postingsFormat;
+ }
+
+ @Override
+ public final FieldInfosFormat fieldInfosFormat() {
+ return fieldInfosFormat;
+ }
+
+ @Override
+ public final SegmentInfoFormat segmentInfoFormat() {
+ return segmentInfosFormat;
+ }
+
+ @Override
+ public final LiveDocsFormat liveDocsFormat() {
+ return liveDocsFormat;
+ }
+
+ @Override
+ public final CompoundFormat compoundFormat() {
+ return compoundFormat;
+ }
+
+ @Override
+ public final PointsFormat pointsFormat() {
+ return pointsFormat;
+ }
+
+ /** Returns the postings format that should be used for writing
+ * new segments of field
.
+ *
+ * The default implementation always returns "Lucene84".
+ *
+ * WARNING: if you subclass, you are responsible for index
+ * backwards compatibility: future version of Lucene are only
+ * guaranteed to be able to read the default implementation.
+ */
+ public PostingsFormat getPostingsFormatForField(String field) {
+ return defaultFormat;
+ }
+
+ /** Returns the docvalues format that should be used for writing
+ * new segments of field
.
+ *
+ * The default implementation always returns "Lucene80".
+ *
+ * WARNING: if you subclass, you are responsible for index
+ * backwards compatibility: future version of Lucene are only
+ * guaranteed to be able to read the default implementation.
+ */
+ public DocValuesFormat getDocValuesFormatForField(String field) {
+ return defaultDVFormat;
+ }
+
+ @Override
+ public final DocValuesFormat docValuesFormat() {
+ return docValuesFormat;
+ }
+
+ private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene80");
+
+ private final NormsFormat normsFormat = new Lucene80NormsFormat();
+
+ @Override
+ public final NormsFormat normsFormat() {
+ return normsFormat;
+ }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene87/Lucene87StoredFieldsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene87/Lucene87StoredFieldsFormat.java
new file mode 100644
index 00000000000..c2bbcedfc74
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene87/Lucene87StoredFieldsFormat.java
@@ -0,0 +1,378 @@
+/*
+ * 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.lucene87;
+
+import java.io.IOException;
+import java.util.Objects;
+import java.util.zip.DataFormatException;
+import java.util.zip.Deflater;
+import java.util.zip.Inflater;
+
+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.index.CorruptIndexException;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.index.StoredFieldVisitor;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.packed.DirectMonotonicWriter;
+
+/**
+ * Lucene 8.7 stored fields format.
+ *
+ *
Principle
+ *
This {@link StoredFieldsFormat} compresses blocks of documents in
+ * order to improve the compression ratio compared to document-level
+ * compression. It uses the LZ4
+ * 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 DEFLATE
+ * 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:
+ *
+ * // 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));
+ *
+ * File formats
+ *
Stored fields are represented by three files:
+ *
+ * -
+ *
A fields data file (extension .fdt
). 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 byte[]
+ * 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
+ * LZ4
+ * compression format.
+ * Notes
+ *
+ * - 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.
+ * - 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.
+ * - In case documents are incompressible, the overhead of the compression format
+ * is less than 0.5%.
+ *
+ *
+ * -
+ *
A fields index file (extension .fdx
). 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.
+ * -
+ *
A fields meta file (extension .fdm
). This file stores metadata
+ * about the monotonic arrays stored in the index file.
+ *
+ *
+ * Known limitations
+ *
This {@link StoredFieldsFormat} does not support individual documents
+ * larger than (231 - 214
) bytes.
+ * @lucene.experimental
+ */
+public class Lucene87StoredFieldsFormat 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 = Lucene87StoredFieldsFormat.class.getSimpleName() + ".mode";
+
+ final Mode mode;
+
+ /** Stored fields format with default options */
+ public Lucene87StoredFieldsFormat() {
+ this(Mode.BEST_SPEED);
+ }
+
+ /** Stored fields format with specified mode */
+ public Lucene87StoredFieldsFormat(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 CompressingStoredFieldsFormat("Lucene87StoredFieldsFastData", CompressionMode.FAST, 16*1024, 128, 10);
+ case BEST_COMPRESSION:
+ return new CompressingStoredFieldsFormat("Lucene87StoredFieldsHighData", BEST_COMPRESSION_MODE, BEST_COMPRESSION_BLOCK_LENGTH, 512, 10);
+ default: throw new AssertionError();
+ }
+ }
+
+ // 8kB seems to be a good trade-off between higher compression rates by not
+ // having to fully bootstrap a dictionary, and indexing rate by not spending
+ // too much CPU initializing data-structures to find strings in this preset
+ // dictionary.
+ private static final int BEST_COMPRESSION_DICT_LENGTH = 8 * 1024;
+ // 48kB seems like a nice trade-off because it's small enough to keep
+ // retrieval fast, yet sub blocks can find strings in a window of 26kB of
+ // data on average (the window grows from 8kB to 32kB in the first 24kB, and
+ // then DEFLATE can use 32kB for the last 24kB) which is close enough to the
+ // maximum window length of DEFLATE of 32kB.
+ private static final int BEST_COMPRESSION_SUB_BLOCK_LENGTH = 48 * 1024;
+ // We shoot for 10 sub blocks per block, which should hopefully amortize the
+ // space overhead of having the first 8kB compressed without any preset dict,
+ // and then remove 8kB in order to avoid creating a tiny 11th sub block if
+ // documents are small.
+ private static final int BEST_COMPRESSION_BLOCK_LENGTH = BEST_COMPRESSION_DICT_LENGTH + 10 * BEST_COMPRESSION_SUB_BLOCK_LENGTH - 8 * 1024;
+
+ /** Compression mode for {@link Mode#BEST_COMPRESSION} */
+ public static final DeflateWithPresetDict BEST_COMPRESSION_MODE = new DeflateWithPresetDict(BEST_COMPRESSION_DICT_LENGTH, BEST_COMPRESSION_SUB_BLOCK_LENGTH);
+
+ /**
+ * 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.
+ */
+ public static class DeflateWithPresetDict extends CompressionMode {
+
+ private final int dictLength, subBlockLength;
+
+ /** Sole constructor. */
+ public DeflateWithPresetDict(int dictLength, int subBlockLength) {
+ this.dictLength = dictLength;
+ this.subBlockLength = subBlockLength;
+ }
+
+ @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, dictLength, subBlockLength);
+ }
+
+ @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 int dictLength;
+ final int blockLength;
+ final Deflater compressor;
+ byte[] compressed;
+ boolean closed;
+
+ DeflateWithPresetDictCompressor(int level, int dictLength, int blockLength) {
+ compressor = new Deflater(level, true);
+ compressed = new byte[64];
+ this.dictLength = dictLength;
+ this.blockLength = blockLength;
+ }
+
+ 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 = Math.min(this.dictLength, len);
+ 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();
+ compressor.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;
+ }
+ }
+ }
+
+}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene87/package-info.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene87/package-info.java
new file mode 100644
index 00000000000..75facdb2fb4
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene87/package-info.java
@@ -0,0 +1,416 @@
+/*
+ * 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.
+ */
+
+/**
+ * Lucene 8.7 file format.
+ *
+ *
Apache Lucene - Index File Formats
+ *
+ *
+ * Introduction
+ *
+ *
This document defines the index file formats used in this version of Lucene.
+ * If you are using a different version of Lucene, please consult the copy of
+ * docs/
that was distributed with
+ * the version you are using.
+ *
This document attempts to provide a high-level definition of the Apache
+ * Lucene file formats.
+ *
+ *
+ * Definitions
+ *
+ *
The fundamental concepts in Lucene are index, document, field and term.
+ *
An index contains a sequence of documents.
+ *
+ * - A document is a sequence of fields.
+ * - A field is a named sequence of terms.
+ * - A term is a sequence of bytes.
+ *
+ *
The same sequence of bytes in two different fields is considered a different
+ * term. Thus terms are represented as a pair: the string naming the field, and the
+ * bytes within the field.
+ *
+ *
Inverted Indexing
+ *
The index stores statistics about terms in order to make term-based search
+ * more efficient. Lucene's index falls into the family of indexes known as an
+ * inverted index. This is because it can list, for a term, the documents
+ * that contain it. This is the inverse of the natural relationship, in which
+ * documents list terms.
+ *
+ *
Types of Fields
+ *
In Lucene, fields may be stored, in which case their text is stored
+ * in the index literally, in a non-inverted manner. Fields that are inverted are
+ * called indexed. A field may be both stored and indexed.
+ *
The text of a field may be tokenized into terms to be indexed, or the
+ * text of a field may be used literally as a term to be indexed. Most fields are
+ * tokenized, but sometimes it is useful for certain identifier fields to be
+ * indexed literally.
+ *
See the {@link org.apache.lucene.document.Field Field}
+ * java docs for more information on Fields.
+ *
+ *
Segments
+ *
Lucene indexes may be composed of multiple sub-indexes, or segments.
+ * Each segment is a fully independent index, which could be searched separately.
+ * Indexes evolve by:
+ *
+ * - Creating new segments for newly added documents.
+ * - Merging existing segments.
+ *
+ *
Searches may involve multiple segments and/or multiple indexes, each index
+ * potentially composed of a set of segments.
+ *
+ *
Document Numbers
+ *
Internally, Lucene refers to documents by an integer document number.
+ * The first document added to an index is numbered zero, and each subsequent
+ * document added gets a number one greater than the previous.
+ *
Note that a document's number may change, so caution should be taken when
+ * storing these numbers outside of Lucene. In particular, numbers may change in
+ * the following situations:
+ *
+ * -
+ *
The numbers stored in each segment are unique only within the segment, and
+ * must be converted before they can be used in a larger context. The standard
+ * technique is to allocate each segment a range of values, based on the range of
+ * numbers used in that segment. To convert a document number from a segment to an
+ * external value, the segment's base document number is added. To convert
+ * an external value back to a segment-specific value, the segment is identified
+ * by the range that the external value is in, and the segment's base value is
+ * subtracted. For example two five document segments might be combined, so that
+ * the first segment has a base value of zero, and the second of five. Document
+ * three from the second segment would have an external value of eight.
+ *
+ * -
+ *
When documents are deleted, gaps are created in the numbering. These are
+ * eventually removed as the index evolves through merging. Deleted documents are
+ * dropped when segments are merged. A freshly-merged segment thus has no gaps in
+ * its numbering.
+ *
+ *
+ *
+ *
+ * Index Structure Overview
+ *
+ *
Each segment index maintains the following:
+ *
+ * -
+ * {@link org.apache.lucene.codecs.lucene86.Lucene86SegmentInfoFormat Segment info}.
+ * This contains metadata about a segment, such as the number of documents,
+ * what files it uses, and information about how the segment is sorted
+ *
+ * -
+ * {@link org.apache.lucene.codecs.lucene60.Lucene60FieldInfosFormat Field names}.
+ * This contains the set of field names used in the index.
+ *
+ * -
+ * {@link org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat 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
+ * returned for each hit when searching. This is keyed by document number.
+ *
+ * -
+ * {@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat Term dictionary}.
+ * A dictionary containing all of the terms used in all of the
+ * indexed fields of all of the documents. The dictionary also contains the number
+ * of documents which contain the term, and pointers to the term's frequency and
+ * proximity data.
+ *
+ * -
+ * {@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat Term Frequency data}.
+ * For each term in the dictionary, the numbers of all the
+ * documents that contain that term, and the frequency of the term in that
+ * document, unless frequencies are omitted ({@link org.apache.lucene.index.IndexOptions#DOCS IndexOptions.DOCS})
+ *
+ * -
+ * {@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat Term Proximity data}.
+ * For each term in the dictionary, the positions that the
+ * term occurs in each document. Note that this will not exist if all fields in
+ * all documents omit position data.
+ *
+ * -
+ * {@link org.apache.lucene.codecs.lucene80.Lucene80NormsFormat Normalization factors}.
+ * For each field in each document, a value is stored
+ * that is multiplied into the score for hits on that field.
+ *
+ * -
+ * {@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vectors}.
+ * For each field in each document, the term vector (sometimes
+ * called document vector) may be stored. A term vector consists of term text and
+ * term frequency. To add Term Vectors to your index see the
+ * {@link org.apache.lucene.document.Field Field} constructors
+ *
+ * -
+ * {@link org.apache.lucene.codecs.lucene80.Lucene80DocValuesFormat Per-document values}.
+ * Like stored values, these are also keyed by document
+ * number, but are generally intended to be loaded into main memory for fast
+ * access. Whereas stored values are generally intended for summary results from
+ * searches, per-document values are useful for things like scoring factors.
+ *
+ * -
+ * {@link org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat Live documents}.
+ * An optional file indicating which documents are live.
+ *
+ * -
+ * {@link org.apache.lucene.codecs.lucene86.Lucene86PointsFormat Point values}.
+ * Optional pair of files, recording dimensionally indexed fields, to enable fast
+ * numeric range filtering and large numeric values like BigInteger and BigDecimal (1D)
+ * and geographic shape intersection (2D, 3D).
+ *
+ *
+ *
Details on each of these are provided in their linked pages.
+ *
+ *
+ * File Naming
+ *
+ *
All files belonging to a segment have the same name with varying extensions.
+ * The extensions correspond to the different file formats described below. When
+ * using the Compound File format (default for small segments) these files (except
+ * for the Segment info file, the Lock file, and Deleted documents file) are collapsed
+ * into a single .cfs file (see below for details)
+ *
Typically, all segments in an index are stored in a single directory,
+ * although this is not required.
+ *
File names are never re-used. That is, when any file is saved
+ * to the Directory it is given a never before used filename. This is achieved
+ * using a simple generations approach. For example, the first segments file is
+ * segments_1, then segments_2, etc. The generation is a sequential long integer
+ * represented in alpha-numeric (base 36) form.
+ *
+ *
+ * Summary of File Extensions
+ *
+ *
The following table summarizes the names and extensions of the files in
+ * Lucene:
+ *
+ * lucene filenames by extension
+ *
+ * Name |
+ * Extension |
+ * Brief Description |
+ *
+ *
+ * {@link org.apache.lucene.index.SegmentInfos Segments File} |
+ * segments_N |
+ * Stores information about a commit point |
+ *
+ *
+ * Lock File |
+ * write.lock |
+ * The Write lock prevents multiple IndexWriters from writing to the same
+ * file. |
+ *
+ *
+ * {@link org.apache.lucene.codecs.lucene86.Lucene86SegmentInfoFormat Segment Info} |
+ * .si |
+ * Stores metadata about a segment |
+ *
+ *
+ * {@link org.apache.lucene.codecs.lucene50.Lucene50CompoundFormat Compound File} |
+ * .cfs, .cfe |
+ * An optional "virtual" file consisting of all the other index files for
+ * systems that frequently run out of file handles. |
+ *
+ *
+ * {@link org.apache.lucene.codecs.lucene60.Lucene60FieldInfosFormat Fields} |
+ * .fnm |
+ * Stores information about the fields |
+ *
+ *
+ * {@link org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat Field Index} |
+ * .fdx |
+ * Contains pointers to field data |
+ *
+ *
+ * {@link org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat Field Data} |
+ * .fdt |
+ * The stored fields for documents |
+ *
+ *
+ * {@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat Term Dictionary} |
+ * .tim |
+ * The term dictionary, stores term info |
+ *
+ *
+ * {@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat Term Index} |
+ * .tip |
+ * The index into the Term Dictionary |
+ *
+ *
+ * {@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat Frequencies} |
+ * .doc |
+ * Contains the list of docs which contain each term along with frequency |
+ *
+ *
+ * {@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat Positions} |
+ * .pos |
+ * Stores position information about where a term occurs in the index |
+ *
+ *
+ * {@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat Payloads} |
+ * .pay |
+ * Stores additional per-position metadata information such as character offsets and user payloads |
+ *
+ *
+ * {@link org.apache.lucene.codecs.lucene80.Lucene80NormsFormat Norms} |
+ * .nvd, .nvm |
+ * Encodes length and boost factors for docs and fields |
+ *
+ *
+ * {@link org.apache.lucene.codecs.lucene80.Lucene80DocValuesFormat Per-Document Values} |
+ * .dvd, .dvm |
+ * Encodes additional scoring factors or other per-document information. |
+ *
+ *
+ * {@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Index} |
+ * .tvx |
+ * Stores offset into the document data file |
+ *
+ *
+ * {@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Data} |
+ * .tvd |
+ * Contains term vector data. |
+ *
+ *
+ * {@link org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat Live Documents} |
+ * .liv |
+ * Info about what documents are live |
+ *
+ *
+ * {@link org.apache.lucene.codecs.lucene86.Lucene86PointsFormat Point values} |
+ * .dii, .dim |
+ * Holds indexed points, if any |
+ *
+ *
+ *
+ *
+ * Lock File
+ * The write lock, which is stored in the index directory by default, is named
+ * "write.lock". If the lock directory is different from the index directory then
+ * the write lock will be named "XXXX-write.lock" where XXXX is a unique prefix
+ * derived from the full path to the index directory. When this file is present, a
+ * writer is currently modifying the index (adding or removing documents). This
+ * lock file ensures that only one writer is modifying the index at a time.
+ *
+ * History
+ * Compatibility notes are provided in this document, describing how file
+ * formats have changed from prior versions:
+ *
+ * - In version 2.1, the file format was changed to allow lock-less commits (ie,
+ * no more commit lock). The change is fully backwards compatible: you can open a
+ * pre-2.1 index for searching or adding/deleting of docs. When the new segments
+ * file is saved (committed), it will be written in the new file format (meaning
+ * no specific "upgrade" process is needed). But note that once a commit has
+ * occurred, pre-2.1 Lucene will not be able to read the index.
+ * - In version 2.3, the file format was changed to allow segments to share a
+ * single set of doc store (vectors & stored fields) files. This allows for
+ * faster indexing in certain cases. The change is fully backwards compatible (in
+ * the same way as the lock-less commits change in 2.1).
+ * - In version 2.4, Strings are now written as true UTF-8 byte sequence, not
+ * Java's modified UTF-8. See
+ * LUCENE-510 for details.
+ * - In version 2.9, an optional opaque Map<String,String> CommitUserData
+ * may be passed to IndexWriter's commit methods (and later retrieved), which is
+ * recorded in the segments_N file. See
+ * LUCENE-1382 for details. Also,
+ * diagnostics were added to each segment written recording details about why it
+ * was written (due to flush, merge; which OS/JRE was used; etc.). See issue
+ * LUCENE-1654 for details.
+ * - In version 3.0, compressed fields are no longer written to the index (they
+ * can still be read, but on merge the new segment will write them, uncompressed).
+ * See issue LUCENE-1960
+ * for details.
+ * - In version 3.1, segments records the code version that created them. See
+ * LUCENE-2720 for details.
+ * Additionally segments track explicitly whether or not they have term vectors.
+ * See LUCENE-2811
+ * for details.
+ * - In version 3.2, numeric fields are written as natively to stored fields
+ * file, previously they were stored in text format only.
+ * - In version 3.4, fields can omit position data while still indexing term
+ * frequencies.
+ * - In version 4.0, the format of the inverted index became extensible via
+ * the {@link org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage
+ * ({@code DocValues}) was introduced. Normalization factors need no longer be a
+ * single byte, they can be any {@link org.apache.lucene.index.NumericDocValues NumericDocValues}.
+ * Terms need not be unicode strings, they can be any byte sequence. Term offsets
+ * can optionally be indexed into the postings lists. Payloads can be stored in the
+ * term vectors.
+ * - In version 4.1, the format of the postings list changed to use either
+ * of FOR compression or variable-byte encoding, depending upon the frequency
+ * of the term. Terms appearing only once were changed to inline directly into
+ * the term dictionary. Stored fields are compressed by default.
+ * - In version 4.2, term vectors are compressed by default. DocValues has
+ * a new multi-valued type (SortedSet), that can be used for faceting/grouping/joining
+ * on multi-valued fields.
+ * - In version 4.5, DocValues were extended to explicitly represent missing values.
+ * - In version 4.6, FieldInfos were extended to support per-field DocValues generation, to
+ * allow updating NumericDocValues fields.
+ * - In version 4.8, checksum footers were added to the end of each index file
+ * for improved data integrity. Specifically, the last 8 bytes of every index file
+ * contain the zlib-crc32 checksum of the file.
+ * - In version 4.9, DocValues has a new multi-valued numeric type (SortedNumeric)
+ * that is suitable for faceting/sorting/analytics.
+ *
- In version 5.4, DocValues have been improved to store more information on disk:
+ * addresses for binary fields and ord indexes for multi-valued fields.
+ *
- In version 6.0, Points were added, for multi-dimensional range/distance search.
+ *
- In version 6.2, new Segment info format that reads/writes the index sort, to support index sorting.
+ *
- In version 7.0, DocValues have been improved to better support sparse doc values
+ * thanks to an iterator API.
+ * - In version 8.0, postings have been enhanced to record, for each block of
+ * doc ids, the (term freq, normalization factor) pairs that may trigger the
+ * maximum score of the block. This information is recorded alongside skip data
+ * in order to be able to skip blocks of doc ids if they may not produce high
+ * enough scores.
+ * Additionally doc values and norms has been extended with jump-tables to make access O(1)
+ * instead of O(n), where n is the number of elements to skip when advancing in the data.
+ * - In version 8.4, postings, positions, offsets and payload lengths have move to a more
+ * performant encoding that is vectorized.
+ * - In version 8.6, index sort serialization is delegated to the sorts themselves, to
+ * allow user-defined sorts to be used
+ *
+ *
+ * Limitations
+ *
+ *
Lucene uses a Java int
to refer to
+ * document numbers, and the index file format uses an Int32
+ * on-disk to store document numbers. This is a limitation
+ * of both the index file format and the current implementation. Eventually these
+ * should be replaced with either UInt64
values, or
+ * better yet, {@link org.apache.lucene.store.DataOutput#writeVInt VInt} values which have no limit.
+ *
+ */
+package org.apache.lucene.codecs.lucene87;
diff --git a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
index 2897a8ac7c1..2be0f71cc28 100644
--- a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
+++ b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
@@ -13,4 +13,4 @@
# See the License for the specific language governing permissions and
# limitations under the License.
-org.apache.lucene.codecs.lucene86.Lucene86Codec
+org.apache.lucene.codecs.lucene87.Lucene87Codec
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestLucene80NormsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestLucene80NormsFormat.java
index b6e7268d67f..011d2caddbb 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestLucene80NormsFormat.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestLucene80NormsFormat.java
@@ -18,14 +18,14 @@ package org.apache.lucene.codecs.lucene80;
import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.lucene86.Lucene86Codec;
import org.apache.lucene.index.BaseNormsFormatTestCase;
+import org.apache.lucene.util.TestUtil;
/**
* Tests Lucene80NormsFormat
*/
public class TestLucene80NormsFormat extends BaseNormsFormatTestCase {
- private final Codec codec = new Lucene86Codec();
+ private final Codec codec = TestUtil.getDefaultCodec();
@Override
protected Codec getCodec() {
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene86/TestLucene86PointsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene86/TestLucene86PointsFormat.java
index 8d5ce08f346..9198301c988 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene86/TestLucene86PointsFormat.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene86/TestLucene86PointsFormat.java
@@ -49,7 +49,7 @@ public class TestLucene86PointsFormat extends BasePointsFormatTestCase {
public TestLucene86PointsFormat() {
// standard issue
- Codec defaultCodec = new Lucene86Codec();
+ Codec defaultCodec = TestUtil.getDefaultCodec();
if (random().nextBoolean()) {
// randomize parameters
maxPointsInLeafNode = TestUtil.nextInt(random(), 50, 500);
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene87/TestLucene87StoredFieldsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene87/TestLucene87StoredFieldsFormat.java
new file mode 100644
index 00000000000..5604d412a82
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene87/TestLucene87StoredFieldsFormat.java
@@ -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.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();
+ }
+}
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene87/TestLucene87StoredFieldsFormatHighCompression.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene87/TestLucene87StoredFieldsFormatHighCompression.java
new file mode 100644
index 00000000000..f4ebca6c1ab
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene87/TestLucene87StoredFieldsFormatHighCompression.java
@@ -0,0 +1,80 @@
+/*
+ * 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.lucene87;
+
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat.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;
+
+import com.carrotsearch.randomizedtesting.generators.RandomPicks;
+
+public class TestLucene87StoredFieldsFormatHighCompression extends BaseStoredFieldsFormatTestCase {
+ @Override
+ protected Codec getCodec() {
+ return new Lucene87Codec(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 Lucene87Codec(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 Lucene87Codec(null);
+ });
+
+ expectThrows(NullPointerException.class, () -> {
+ new Lucene87StoredFieldsFormat(null);
+ });
+ }
+}
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene87/TestLucene87StoredFieldsFormatMergeInstance.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene87/TestLucene87StoredFieldsFormatMergeInstance.java
new file mode 100644
index 00000000000..0015fb208e4
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene87/TestLucene87StoredFieldsFormatMergeInstance.java
@@ -0,0 +1,29 @@
+/*
+ * 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.lucene87;
+
+/**
+ * Test the merge instance of the Lucene50 stored fields format.
+ */
+public class TestLucene87StoredFieldsFormatMergeInstance extends TestLucene87StoredFieldsFormat {
+
+ @Override
+ protected boolean shouldTestMergeInstance() {
+ return true;
+ }
+
+}
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java b/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
index d982953a2f6..d937c2f8242 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
@@ -396,7 +396,7 @@ public class TestPointValues extends LuceneTestCase {
public void testDifferentCodecs1() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
- iwc.setCodec(Codec.forName("Lucene86"));
+ iwc.setCodec(TestUtil.getDefaultCodec());
IndexWriter w = new IndexWriter(dir, iwc);
Document doc = new Document();
doc.add(new IntPoint("int", 1));
@@ -427,7 +427,7 @@ public class TestPointValues extends LuceneTestCase {
w.close();
iwc = new IndexWriterConfig(new MockAnalyzer(random()));
- iwc.setCodec(Codec.forName("Lucene86"));
+ iwc.setCodec(TestUtil.getDefaultCodec());
w = new IndexWriter(dir, iwc);
doc = new Document();
doc.add(new IntPoint("int", 1));
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBoolean2.java b/lucene/core/src/test/org/apache/lucene/search/TestBoolean2.java
index 3400f0e6dd6..cac56e98561 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestBoolean2.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestBoolean2.java
@@ -23,7 +23,6 @@ import java.util.Collections;
import java.util.Random;
import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.codecs.Codec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
@@ -96,7 +95,7 @@ public class TestBoolean2 extends LuceneTestCase {
IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()));
// randomized codecs are sometimes too costly for this test:
- iwc.setCodec(Codec.forName("Lucene86"));
+ iwc.setCodec(TestUtil.getDefaultCodec());
iwc.setMergePolicy(newLogMergePolicy());
RandomIndexWriter writer= new RandomIndexWriter(random(), directory, iwc);
// we'll make a ton of docs, disable store/norms/vectors
@@ -141,7 +140,7 @@ public class TestBoolean2 extends LuceneTestCase {
iwc = newIndexWriterConfig(new MockAnalyzer(random()));
// we need docID order to be preserved:
// randomized codecs are sometimes too costly for this test:
- iwc.setCodec(Codec.forName("Lucene86"));
+ iwc.setCodec(TestUtil.getDefaultCodec());
iwc.setMergePolicy(newLogMergePolicy());
try (IndexWriter w = new IndexWriter(singleSegmentDirectory, iwc)) {
w.forceMerge(1, true);
@@ -167,7 +166,7 @@ public class TestBoolean2 extends LuceneTestCase {
iwc = newIndexWriterConfig(new MockAnalyzer(random()));
// randomized codecs are sometimes too costly for this test:
- iwc.setCodec(Codec.forName("Lucene86"));
+ iwc.setCodec(TestUtil.getDefaultCodec());
RandomIndexWriter w = new RandomIndexWriter(random(), dir2, iwc);
w.addIndexes(copy);
copy.close();
@@ -179,7 +178,7 @@ public class TestBoolean2 extends LuceneTestCase {
iwc = newIndexWriterConfig(new MockAnalyzer(random()));
iwc.setMaxBufferedDocs(TestUtil.nextInt(random(), 50, 1000));
// randomized codecs are sometimes too costly for this test:
- iwc.setCodec(Codec.forName("Lucene86"));
+ iwc.setCodec(TestUtil.getDefaultCodec());
RandomIndexWriter w = new RandomIndexWriter(random(), dir2, iwc);
doc = new Document();
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestFloatPointNearestNeighbor.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestFloatPointNearestNeighbor.java
index a14204caf33..f77d5947878 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestFloatPointNearestNeighbor.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestFloatPointNearestNeighbor.java
@@ -18,7 +18,6 @@ package org.apache.lucene.document;
import java.util.Arrays;
-import org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
@@ -243,7 +242,7 @@ public class TestFloatPointNearestNeighbor extends LuceneTestCase {
private IndexWriterConfig getIndexWriterConfig() {
IndexWriterConfig iwc = newIndexWriterConfig();
- iwc.setCodec(Codec.forName("Lucene86"));
+ iwc.setCodec(TestUtil.getDefaultCodec());
return iwc;
}
}
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/TestNearest.java b/lucene/sandbox/src/test/org/apache/lucene/search/TestNearest.java
index a149aceb880..98a3de1e239 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/search/TestNearest.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/TestNearest.java
@@ -19,7 +19,6 @@ package org.apache.lucene.search;
import java.util.Arrays;
import java.util.Comparator;
-import org.apache.lucene.codecs.Codec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.LatLonDocValuesField;
@@ -246,7 +245,7 @@ public class TestNearest extends LuceneTestCase {
private IndexWriterConfig getIndexWriterConfig() {
IndexWriterConfig iwc = newIndexWriterConfig();
- iwc.setCodec(Codec.forName("Lucene86"));
+ iwc.setCodec(TestUtil.getDefaultCodec());
return iwc;
}
}
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
index 12c8902f4ad..f4a7c9912b8 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
@@ -39,7 +39,7 @@ import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene86.Lucene86Codec;
+import org.apache.lucene.codecs.lucene87.Lucene87Codec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.IntPoint;
@@ -887,7 +887,7 @@ public class TestSuggestField extends LuceneTestCase {
static IndexWriterConfig iwcWithSuggestField(Analyzer analyzer, final Set suggestFields) {
IndexWriterConfig iwc = newIndexWriterConfig(random(), analyzer);
iwc.setMergePolicy(newLogMergePolicy());
- Codec filterCodec = new Lucene86Codec() {
+ Codec filterCodec = new Lucene87Codec() {
CompletionPostingsFormat.FSTLoadMode fstLoadMode =
RandomPicks.randomFrom(random(), CompletionPostingsFormat.FSTLoadMode.values());
PostingsFormat postingsFormat = new Completion84PostingsFormat(fstLoadMode);
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/CompressingCodec.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/CompressingCodec.java
index 4f334ae1205..9fd243fcd6a 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/CompressingCodec.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/CompressingCodec.java
@@ -37,7 +37,7 @@ public abstract class CompressingCodec extends FilterCodec {
* Create a random instance.
*/
public static CompressingCodec randomInstance(Random random, int chunkSize, int maxDocsPerChunk, boolean withSegmentSuffix, int blockShift) {
- switch (random.nextInt(4)) {
+ switch (random.nextInt(5)) {
case 0:
return new FastCompressingCodec(chunkSize, maxDocsPerChunk, withSegmentSuffix, blockShift);
case 1:
@@ -46,6 +46,8 @@ public abstract class CompressingCodec extends FilterCodec {
return new HighCompressionCompressingCodec(chunkSize, maxDocsPerChunk, withSegmentSuffix, blockShift);
case 3:
return new DummyCompressingCodec(chunkSize, maxDocsPerChunk, withSegmentSuffix, blockShift);
+ case 4:
+ return new DeflateWithPresetCompressingCodec(chunkSize, maxDocsPerChunk, withSegmentSuffix, blockShift);
default:
throw new AssertionError();
}
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/DeflateWithPresetCompressingCodec.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/DeflateWithPresetCompressingCodec.java
new file mode 100644
index 00000000000..9d1791e5f80
--- /dev/null
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/DeflateWithPresetCompressingCodec.java
@@ -0,0 +1,36 @@
+/*
+ * 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 org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat.DeflateWithPresetDict;
+
+/** CompressionCodec that uses {@link DeflateWithPresetDict}. */
+public class DeflateWithPresetCompressingCodec extends CompressingCodec {
+
+ /** Constructor that allows to configure the chunk size. */
+ public DeflateWithPresetCompressingCodec(int chunkSize, int maxDocsPerChunk, boolean withSegmentSuffix, int blockSize) {
+ super("DeflateWithPresetCompressingStoredFieldsData",
+ withSegmentSuffix ? "DeflateWithPresetCompressingStoredFields" : "",
+ new DeflateWithPresetDict(chunkSize/10, chunkSize/3+1), chunkSize, maxDocsPerChunk, blockSize);
+ }
+
+ /** No-arg constructor. */
+ public DeflateWithPresetCompressingCodec() {
+ this(1<<18, 512, false, 10);
+ }
+
+}
diff --git a/lucene/test-framework/src/java/org/apache/lucene/geo/BaseGeoPointTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/geo/BaseGeoPointTestCase.java
index f556c0d55cc..c080db19b08 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/geo/BaseGeoPointTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/geo/BaseGeoPointTestCase.java
@@ -26,6 +26,7 @@ import java.util.Locale;
import java.util.Set;
import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PointsReader;
@@ -1276,7 +1277,8 @@ public abstract class BaseGeoPointTestCase extends LuceneTestCase {
// Else seeds may not reproduce:
iwc.setMergeScheduler(new SerialMergeScheduler());
int pointsInLeaf = 2 + random().nextInt(4);
- iwc.setCodec(new FilterCodec("Lucene86", TestUtil.getDefaultCodec()) {
+ final Codec in = TestUtil.getDefaultCodec();
+ iwc.setCodec(new FilterCodec(in.getName(), in) {
@Override
public PointsFormat pointsFormat() {
return new PointsFormat() {
diff --git a/lucene/test-framework/src/java/org/apache/lucene/geo/BaseXYPointTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/geo/BaseXYPointTestCase.java
index f60bd4c07d3..c9240d7e7ae 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/geo/BaseXYPointTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/geo/BaseXYPointTestCase.java
@@ -26,6 +26,7 @@ import java.util.Locale;
import java.util.Set;
import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PointsReader;
@@ -1190,7 +1191,8 @@ public abstract class BaseXYPointTestCase extends LuceneTestCase {
// Else seeds may not reproduce:
iwc.setMergeScheduler(new SerialMergeScheduler());
int pointsInLeaf = 2 + random().nextInt(4);
- iwc.setCodec(new FilterCodec("Lucene86", TestUtil.getDefaultCodec()) {
+ Codec in = TestUtil.getDefaultCodec();
+ iwc.setCodec(new FilterCodec(in.getName(), in) {
@Override
public PointsFormat pointsFormat() {
return new PointsFormat() {
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java b/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java
index aef11ac1d7f..81cb328aada 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java
@@ -33,8 +33,8 @@ import org.apache.lucene.codecs.asserting.AssertingDocValuesFormat;
import org.apache.lucene.codecs.asserting.AssertingPostingsFormat;
import org.apache.lucene.codecs.cheapbastard.CheapBastardCodec;
import org.apache.lucene.codecs.compressing.CompressingCodec;
-import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
-import org.apache.lucene.codecs.lucene86.Lucene86Codec;
+import org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat;
+import org.apache.lucene.codecs.lucene87.Lucene87Codec;
import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
import org.apache.lucene.index.RandomCodec;
@@ -187,8 +187,8 @@ final class TestRuleSetupAndRestoreClassEnv extends AbstractBeforeAfterRule {
codec = new AssertingCodec();
} else if ("Compressing".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 6 && !shouldAvoidCodec("Compressing"))) {
codec = CompressingCodec.randomInstance(random);
- } else if ("Lucene84".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 5 && !shouldAvoidCodec("Lucene84"))) {
- codec = new Lucene86Codec(RandomPicks.randomFrom(random, Lucene50StoredFieldsFormat.Mode.values())
+ } else if ("Lucene87".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 5 && !shouldAvoidCodec("Lucene87"))) {
+ codec = new Lucene87Codec(RandomPicks.randomFrom(random, Lucene87StoredFieldsFormat.Mode.values())
);
} else if (!"random".equals(TEST_CODEC)) {
codec = Codec.forName(TEST_CODEC);
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
index 2dc9ead733f..7104a85c242 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
@@ -54,7 +54,7 @@ import org.apache.lucene.codecs.blockterms.LuceneFixedGap;
import org.apache.lucene.codecs.blocktreeords.BlockTreeOrdsPostingsFormat;
import org.apache.lucene.codecs.lucene80.Lucene80DocValuesFormat;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat;
-import org.apache.lucene.codecs.lucene86.Lucene86Codec;
+import org.apache.lucene.codecs.lucene87.Lucene87Codec;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
import org.apache.lucene.document.BinaryDocValuesField;
@@ -919,7 +919,7 @@ public final class TestUtil {
* This may be different than {@link Codec#getDefault()} because that is randomized.
*/
public static Codec getDefaultCodec() {
- return new Lucene86Codec();
+ return new Lucene87Codec();
}
/**
diff --git a/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec b/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
index 282f5dd20c8..5892cb05f90 100644
--- a/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
+++ b/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
@@ -15,6 +15,7 @@
org.apache.lucene.codecs.asserting.AssertingCodec
org.apache.lucene.codecs.cheapbastard.CheapBastardCodec
+org.apache.lucene.codecs.compressing.DeflateWithPresetCompressingCodec
org.apache.lucene.codecs.compressing.FastCompressingCodec
org.apache.lucene.codecs.compressing.FastDecompressionCompressingCodec
org.apache.lucene.codecs.compressing.HighCompressionCompressingCodec
diff --git a/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java b/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java
index 6fc3629ad1e..edad01ebae3 100644
--- a/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java
@@ -23,8 +23,8 @@ import java.util.Locale;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
-import org.apache.lucene.codecs.lucene86.Lucene86Codec;
+import org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat.Mode;
+import org.apache.lucene.codecs.lucene87.Lucene87Codec;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.util.NamedList;
@@ -92,7 +92,7 @@ public class SchemaCodecFactory extends CodecFactory implements SolrCoreAware {
compressionMode = SOLR_DEFAULT_COMPRESSION_MODE;
log.debug("Using default compressionMode: {}", compressionMode);
}
- codec = new Lucene86Codec(compressionMode) {
+ codec = new Lucene87Codec(compressionMode) {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
final SchemaField schemaField = core.getLatestSchema().getFieldOrNull(field);