From 4d7b2aebfe01e2a8d0407c179238bb5e092b8d2a Mon Sep 17 00:00:00 2001 From: Julie Tibshirani Date: Mon, 22 Feb 2021 11:49:02 -0800 Subject: [PATCH] LUCENE-9705: Create Lucene90DocValuesFormat and Lucene90NormsFormat (#2392) For now these are just copies of Lucene80DocValuesFormat and Lucene80NormsFormat. The existing formats were moved to backwards-codecs. --- .../lucene60/Lucene60FieldInfosFormat.java | 2 +- .../backward_codecs/lucene80/IndexedDISI.java | 708 +++++++ .../lucene80/Lucene80Codec.java | 3 +- .../lucene80/Lucene80DocValuesConsumer.java | 16 +- .../lucene80/Lucene80DocValuesFormat.java | 7 +- .../lucene80/Lucene80DocValuesProducer.java | 2 +- .../lucene80/Lucene80NormsFormat.java | 13 +- .../lucene80/Lucene80NormsProducer.java | 6 +- .../lucene80/package-info.java | 2 +- .../lucene84/Lucene84Codec.java | 4 +- .../lucene86/Lucene86Codec.java | 6 +- .../lucene87/Lucene87Codec.java | 6 +- .../org.apache.lucene.codecs.DocValuesFormat | 1 + .../Lucene87/Lucene87RWCodec.java | 10 +- .../lucene50/TestLucene50CompoundFormat.java | 2 +- .../BaseLucene80DocValuesFormatTestCase.java | 2 +- .../lucene80/Lucene80NormsConsumer.java | 4 +- .../lucene80/Lucene80RWNormsFormat.java | 30 + ...estCompressionLucene80DocValuesFormat.java | 2 +- .../TestBestSpeedLucene80DocValuesFormat.java | 2 +- .../lucene80/TestDocValuesCompression.java | 12 +- .../lucene80/TestIndexedDISI.java | 593 ++++++ .../lucene80/TestLucene80NormsFormat.java | 6 +- .../TestLucene80NormsFormatMergeInstance.java | 2 +- .../lucene84/Lucene84RWCodec.java | 7 + .../lucene86/Lucene86RWCodec.java | 7 + .../{lucene80 => lucene90}/IndexedDISI.java | 2 +- .../lucene/codecs/lucene90/Lucene90Codec.java | 14 +- .../lucene90/Lucene90DocValuesConsumer.java | 1036 ++++++++++ .../lucene90/Lucene90DocValuesFormat.java | 216 ++ .../lucene90/Lucene90DocValuesProducer.java | 1734 +++++++++++++++++ .../lucene90/Lucene90NormsConsumer.java | 176 ++ .../codecs/lucene90/Lucene90NormsFormat.java | 106 + .../lucene90/Lucene90NormsProducer.java | 492 +++++ .../lucene/codecs/lucene90/package-info.java | 8 +- .../org.apache.lucene.codecs.DocValuesFormat | 2 +- .../lucene80/TestLucene90NormsFormat.java | 31 + .../TestLucene90NormsFormatMergeInstance.java | 26 + .../BaseLucene90DocValuesFormatTestCase.java | 764 ++++++++ ...estCompressionLucene90DocValuesFormat.java | 33 + .../TestBestSpeedLucene90DocValuesFormat.java | 32 + .../lucene90/TestDocValuesCompression.java | 313 +++ .../TestIndexedDISI.java | 2 +- .../org/apache/lucene/index/RandomCodec.java | 4 +- .../java/org/apache/lucene/util/TestUtil.java | 4 +- 45 files changed, 6383 insertions(+), 67 deletions(-) create mode 100644 lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene80/IndexedDISI.java rename lucene/{core/src/java/org/apache/lucene/codecs => backward-codecs/src/java/org/apache/lucene/backward_codecs}/lucene80/Lucene80DocValuesConsumer.java (98%) rename lucene/{core/src/java/org/apache/lucene/codecs => backward-codecs/src/java/org/apache/lucene/backward_codecs}/lucene80/Lucene80DocValuesFormat.java (97%) rename lucene/{core/src/java/org/apache/lucene/codecs => backward-codecs/src/java/org/apache/lucene/backward_codecs}/lucene80/Lucene80DocValuesProducer.java (99%) rename lucene/{core/src/java/org/apache/lucene/codecs => backward-codecs/src/java/org/apache/lucene/backward_codecs}/lucene80/Lucene80NormsFormat.java (91%) rename lucene/{core/src/java/org/apache/lucene/codecs => backward-codecs/src/java/org/apache/lucene/backward_codecs}/lucene80/Lucene80NormsProducer.java (98%) rename lucene/{core/src/test/org/apache/lucene/codecs => backward-codecs/src/test/org/apache/lucene/backward_codecs}/lucene80/BaseLucene80DocValuesFormatTestCase.java (99%) rename lucene/{core/src/java/org/apache/lucene/codecs => backward-codecs/src/test/org/apache/lucene/backward_codecs}/lucene80/Lucene80NormsConsumer.java (97%) create mode 100644 lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/Lucene80RWNormsFormat.java rename lucene/{core/src/test/org/apache/lucene/codecs => backward-codecs/src/test/org/apache/lucene/backward_codecs}/lucene80/TestBestCompressionLucene80DocValuesFormat.java (96%) rename lucene/{core/src/test/org/apache/lucene/codecs => backward-codecs/src/test/org/apache/lucene/backward_codecs}/lucene80/TestBestSpeedLucene80DocValuesFormat.java (95%) rename lucene/{core/src/test/org/apache/lucene/codecs => backward-codecs/src/test/org/apache/lucene/backward_codecs}/lucene80/TestDocValuesCompression.java (97%) create mode 100644 lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/TestIndexedDISI.java rename lucene/{core/src/test/org/apache/lucene/codecs => backward-codecs/src/test/org/apache/lucene/backward_codecs}/lucene80/TestLucene80NormsFormat.java (86%) rename lucene/{core/src/test/org/apache/lucene/codecs => backward-codecs/src/test/org/apache/lucene/backward_codecs}/lucene80/TestLucene80NormsFormatMergeInstance.java (95%) rename lucene/core/src/java/org/apache/lucene/codecs/{lucene80 => lucene90}/IndexedDISI.java (99%) create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesConsumer.java create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesFormat.java create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesProducer.java create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90NormsConsumer.java create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90NormsFormat.java create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90NormsProducer.java create mode 100644 lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestLucene90NormsFormat.java create mode 100644 lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestLucene90NormsFormatMergeInstance.java create mode 100644 lucene/core/src/test/org/apache/lucene/codecs/lucene90/BaseLucene90DocValuesFormatTestCase.java create mode 100644 lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestBestCompressionLucene90DocValuesFormat.java create mode 100644 lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestBestSpeedLucene90DocValuesFormat.java create mode 100644 lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestDocValuesCompression.java rename lucene/core/src/test/org/apache/lucene/codecs/{lucene80 => lucene90}/TestIndexedDISI.java (99%) diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene60/Lucene60FieldInfosFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene60/Lucene60FieldInfosFormat.java index 3eb352e41c0..929569383d0 100644 --- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene60/Lucene60FieldInfosFormat.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene60/Lucene60FieldInfosFormat.java @@ -314,7 +314,7 @@ public final class Lucene60FieldInfosFormat extends FieldInfosFormat { /** * Note: although this format is only used on older versions, we need to keep the write logic in * addition to the read logic. It's possible for field infos on older segments to be written to, - * for example as part of in-place doc values updates. + * for example as part of doc values updates. */ @Override public void write( diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene80/IndexedDISI.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene80/IndexedDISI.java new file mode 100644 index 00000000000..5e370b5d44b --- /dev/null +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene80/IndexedDISI.java @@ -0,0 +1,708 @@ +/* + * 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.lucene80; + +import java.io.DataInput; +import java.io.IOException; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.store.RandomAccessInput; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.BitSetIterator; +import org.apache.lucene.util.FixedBitSet; +import org.apache.lucene.util.RoaringDocIdSet; + +/** + * Disk-based implementation of a {@link DocIdSetIterator} which can return the index of the current + * document, i.e. the ordinal of the current document among the list of documents that this iterator + * can return. This is useful to implement sparse doc values by only having to encode values for + * documents that actually have a value. + * + *

Implementation-wise, this {@link DocIdSetIterator} is inspired of {@link RoaringDocIdSet + * roaring bitmaps} and encodes ranges of {@code 65536} documents independently and picks between 3 + * encodings depending on the density of the range: + * + *

+ * + *

Only ranges that contain at least one value are encoded. + * + *

This implementation uses 6 bytes per document in the worst-case, which happens in the case + * that all ranges contain exactly one document. + * + *

To avoid O(n) lookup time complexity, with n being the number of documents, two lookup tables + * are used: A lookup table for block offset and index, and a rank structure for DENSE block index + * lookups. + * + *

The lookup table is an array of {@code int}-pairs, with a pair for each block. It allows for + * direct jumping to the block, as opposed to iteration from the current position and forward one + * block at a time. + * + *

Each int-pair entry consists of 2 logical parts: + * + *

The first 32 bit int holds the index (number of set bits in the blocks) up to just before the + * wanted block. The maximum number of set bits is the maximum number of documents, which is < 2^31. + * + *

The next int holds the offset in bytes into the underlying slice. As there is a maximum of + * 2^16 blocks, it follows that the maximum size of any block must not exceed 2^15 bytes to avoid + * overflow (2^16 bytes if the int is treated as unsigned). This is currently the case, with the + * largest block being DENSE and using 2^13 + 36 bytes. + * + *

The cache overhead is numDocs/1024 bytes. + * + *

Note: There are 4 types of blocks: ALL, DENSE, SPARSE and non-existing (0 set bits). In the + * case of non-existing blocks, the entry in the lookup table has index equal to the previous entry + * and offset equal to the next non-empty block. + * + *

The block lookup table is stored at the end of the total block structure. + * + *

The rank structure for DENSE blocks is an array of byte-pairs with an entry for each sub-block + * (default 512 bits) out of the 65536 bits in the outer DENSE block. + * + *

Each rank-entry states the number of set bits within the block up to the bit before the bit + * positioned at the start of the sub-block. Note that that the rank entry of the first sub-block is + * always 0 and that the last entry can at most be 65536-2 = 65634 and thus will always fit into an + * byte-pair of 16 bits. + * + *

The rank structure for a given DENSE block is stored at the beginning of the DENSE block. This + * ensures locality and keeps logistics simple. + * + * @lucene.internal + */ +final class IndexedDISI extends DocIdSetIterator { + + // jump-table time/space trade-offs to consider: + // The block offsets and the block indexes could be stored in more compressed form with + // two PackedInts or two MonotonicDirectReaders. + // The DENSE ranks (default 128 shorts = 256 bytes) could likewise be compressed. But as there is + // at least 4096 set bits in DENSE blocks, there will be at least one rank with 2^12 bits, so it + // is doubtful if there is much to gain here. + + private static final int BLOCK_SIZE = + 65536; // The number of docIDs that a single block represents + + private static final int DENSE_BLOCK_LONGS = BLOCK_SIZE / Long.SIZE; // 1024 + public static final byte DEFAULT_DENSE_RANK_POWER = 9; // Every 512 docIDs / 8 longs + + static final int MAX_ARRAY_LENGTH = (1 << 12) - 1; + + private static void flush( + int block, FixedBitSet buffer, int cardinality, byte denseRankPower, IndexOutput out) + throws IOException { + assert block >= 0 && block < 65536; + out.writeShort((short) block); + assert cardinality > 0 && cardinality <= 65536; + out.writeShort((short) (cardinality - 1)); + if (cardinality > MAX_ARRAY_LENGTH) { + if (cardinality != 65536) { // all docs are set + if (denseRankPower != -1) { + final byte[] rank = createRank(buffer, denseRankPower); + out.writeBytes(rank, rank.length); + } + for (long word : buffer.getBits()) { + out.writeLong(word); + } + } + } else { + BitSetIterator it = new BitSetIterator(buffer, cardinality); + for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) { + out.writeShort((short) doc); + } + } + } + + // Creates a DENSE rank-entry (the number of set bits up to a given point) for the buffer. + // One rank-entry for every {@code 2^denseRankPower} bits, with each rank-entry using 2 bytes. + // Represented as a byte[] for fast flushing and mirroring of the retrieval representation. + private static byte[] createRank(FixedBitSet buffer, byte denseRankPower) { + final int longsPerRank = 1 << (denseRankPower - 6); + final int rankMark = longsPerRank - 1; + final int rankIndexShift = denseRankPower - 7; // 6 for the long (2^6) + 1 for 2 bytes/entry + final byte[] rank = new byte[DENSE_BLOCK_LONGS >> rankIndexShift]; + final long[] bits = buffer.getBits(); + int bitCount = 0; + for (int word = 0; word < DENSE_BLOCK_LONGS; word++) { + if ((word & rankMark) == 0) { // Every longsPerRank longs + rank[word >> rankIndexShift] = (byte) (bitCount >> 8); + rank[(word >> rankIndexShift) + 1] = (byte) (bitCount & 0xFF); + } + bitCount += Long.bitCount(bits[word]); + } + return rank; + } + + /** + * Writes the docIDs from it to out, in logical blocks, one for each 65536 docIDs in monotonically + * increasing gap-less order. DENSE blocks uses {@link #DEFAULT_DENSE_RANK_POWER} of 9 (every 512 + * docIDs / 8 longs). The caller must keep track of the number of jump-table entries (returned by + * this method) as well as the denseRankPower (9 for this method) and provide them when + * constructing an IndexedDISI for reading. + * + * @param it the document IDs. + * @param out destination for the blocks. + * @throws IOException if there was an error writing to out. + * @return the number of jump-table entries following the blocks, -1 for no entries. This should + * be stored in meta and used when creating an instance of IndexedDISI. + */ + static short writeBitSet(DocIdSetIterator it, IndexOutput out) throws IOException { + return writeBitSet(it, out, DEFAULT_DENSE_RANK_POWER); + } + + /** + * Writes the docIDs from it to out, in logical blocks, one for each 65536 docIDs in monotonically + * increasing gap-less order. The caller must keep track of the number of jump-table entries + * (returned by this method) as well as the denseRankPower and provide them when constructing an + * IndexedDISI for reading. + * + * @param it the document IDs. + * @param out destination for the blocks. + * @param denseRankPower for {@link Method#DENSE} blocks, a rank will be written every {@code + * 2^denseRankPower} docIDs. Values < 7 (every 128 docIDs) or > 15 (every 32768 docIDs) + * disables DENSE rank. Recommended values are 8-12: Every 256-4096 docIDs or 4-64 longs. + * {@link #DEFAULT_DENSE_RANK_POWER} is 9: Every 512 docIDs. This should be stored in meta and + * used when creating an instance of IndexedDISI. + * @throws IOException if there was an error writing to out. + * @return the number of jump-table entries following the blocks, -1 for no entries. This should + * be stored in meta and used when creating an instance of IndexedDISI. + */ + static short writeBitSet(DocIdSetIterator it, IndexOutput out, byte denseRankPower) + throws IOException { + final long origo = out.getFilePointer(); // All jumps are relative to the origo + if ((denseRankPower < 7 || denseRankPower > 15) && denseRankPower != -1) { + throw new IllegalArgumentException( + "Acceptable values for denseRankPower are 7-15 (every 128-32768 docIDs). " + + "The provided power was " + + denseRankPower + + " (every " + + (int) Math.pow(2, denseRankPower) + + " docIDs)"); + } + int totalCardinality = 0; + int blockCardinality = 0; + final FixedBitSet buffer = new FixedBitSet(1 << 16); + int[] jumps = new int[ArrayUtil.oversize(1, Integer.BYTES * 2)]; + int prevBlock = -1; + int jumpBlockIndex = 0; + + for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) { + final int block = doc >>> 16; + if (prevBlock != -1 && block != prevBlock) { + // Track offset+index from previous block up to current + jumps = + addJumps( + jumps, + out.getFilePointer() - origo, + totalCardinality, + jumpBlockIndex, + prevBlock + 1); + jumpBlockIndex = prevBlock + 1; + // Flush block + flush(prevBlock, buffer, blockCardinality, denseRankPower, out); + // Reset for next block + buffer.clear(0, buffer.length()); + totalCardinality += blockCardinality; + blockCardinality = 0; + } + buffer.set(doc & 0xFFFF); + blockCardinality++; + prevBlock = block; + } + if (blockCardinality > 0) { + jumps = + addJumps( + jumps, out.getFilePointer() - origo, totalCardinality, jumpBlockIndex, prevBlock + 1); + totalCardinality += blockCardinality; + flush(prevBlock, buffer, blockCardinality, denseRankPower, out); + buffer.clear(0, buffer.length()); + prevBlock++; + } + final int lastBlock = + prevBlock == -1 ? 0 : prevBlock; // There will always be at least 1 block (NO_MORE_DOCS) + // Last entry is a SPARSE with blockIndex == 32767 and the single entry 65535, which becomes the + // docID NO_MORE_DOCS + // To avoid creating 65K jump-table entries, only a single entry is created pointing to the + // offset of the + // NO_MORE_DOCS block, with the jumpBlockIndex set to the logical EMPTY block after all real + // blocks. + jumps = + addJumps(jumps, out.getFilePointer() - origo, totalCardinality, lastBlock, lastBlock + 1); + buffer.set(DocIdSetIterator.NO_MORE_DOCS & 0xFFFF); + flush(DocIdSetIterator.NO_MORE_DOCS >>> 16, buffer, 1, denseRankPower, out); + // offset+index jump-table stored at the end + return flushBlockJumps(jumps, lastBlock + 1, out); + } + + // Adds entries to the offset & index jump-table for blocks + private static int[] addJumps(int[] jumps, long offset, int index, int startBlock, int endBlock) { + assert offset < Integer.MAX_VALUE + : "Logically the offset should not exceed 2^30 but was >= Integer.MAX_VALUE"; + jumps = ArrayUtil.grow(jumps, (endBlock + 1) * 2); + for (int b = startBlock; b < endBlock; b++) { + jumps[b * 2] = index; + jumps[b * 2 + 1] = (int) offset; + } + return jumps; + } + + // Flushes the offset & index jump-table for blocks. This should be the last data written to out + // This method returns the blockCount for the blocks reachable for the jump_table or -1 for no + // jump-table + private static short flushBlockJumps(int[] jumps, int blockCount, IndexOutput out) + throws IOException { + if (blockCount + == 2) { // Jumps with a single real entry + NO_MORE_DOCS is just wasted space so we ignore + // that + blockCount = 0; + } + for (int i = 0; i < blockCount; i++) { + out.writeInt(jumps[i * 2]); // index + out.writeInt(jumps[i * 2 + 1]); // offset + } + // As there are at most 32k blocks, the count is a short + // The jumpTableOffset will be at lastPos - (blockCount * Long.BYTES) + return (short) blockCount; + } + + // Members are pkg-private to avoid synthetic accessors when accessed from the `Method` enum + + /** The slice that stores the {@link DocIdSetIterator}. */ + final IndexInput slice; + + final int jumpTableEntryCount; + final byte denseRankPower; + final RandomAccessInput jumpTable; // Skip blocks of 64K bits + final byte[] denseRankTable; + final long cost; + + /** + * This constructor always creates a new blockSlice and a new jumpTable from in, to ensure that + * operations are independent from the caller. See {@link #IndexedDISI(IndexInput, + * RandomAccessInput, int, byte, long)} for re-use of blockSlice and jumpTable. + * + * @param in backing data. + * @param offset starting offset for blocks in the backing data. + * @param length the number of bytes holding blocks and jump-table in the backing data. + * @param jumpTableEntryCount the number of blocks covered by the jump-table. This must match the + * number returned by {@link #writeBitSet(DocIdSetIterator, IndexOutput, byte)}. + * @param denseRankPower the number of docIDs covered by each rank entry in DENSE blocks, + * expressed as {@code 2^denseRankPower}. This must match the power given in {@link + * #writeBitSet(DocIdSetIterator, IndexOutput, byte)} + * @param cost normally the number of logical docIDs. + */ + IndexedDISI( + IndexInput in, + long offset, + long length, + int jumpTableEntryCount, + byte denseRankPower, + long cost) + throws IOException { + this( + createBlockSlice(in, "docs", offset, length, jumpTableEntryCount), + createJumpTable(in, offset, length, jumpTableEntryCount), + jumpTableEntryCount, + denseRankPower, + cost); + } + + /** + * This constructor allows to pass the slice and jumpTable directly in case it helps reuse. see + * eg. Lucene80 norms producer's merge instance. + * + * @param blockSlice data blocks, normally created by {@link #createBlockSlice}. + * @param jumpTable table holding jump-data for block-skips, normally created by {@link + * #createJumpTable}. + * @param jumpTableEntryCount the number of blocks covered by the jump-table. This must match the + * number returned by {@link #writeBitSet(DocIdSetIterator, IndexOutput, byte)}. + * @param denseRankPower the number of docIDs covered by each rank entry in DENSE blocks, + * expressed as {@code 2^denseRankPower}. This must match the power given in {@link + * #writeBitSet(DocIdSetIterator, IndexOutput, byte)} + * @param cost normally the number of logical docIDs. + */ + IndexedDISI( + IndexInput blockSlice, + RandomAccessInput jumpTable, + int jumpTableEntryCount, + byte denseRankPower, + long cost) + throws IOException { + if ((denseRankPower < 7 || denseRankPower > 15) && denseRankPower != -1) { + throw new IllegalArgumentException( + "Acceptable values for denseRankPower are 7-15 (every 128-32768 docIDs). " + + "The provided power was " + + denseRankPower + + " (every " + + (int) Math.pow(2, denseRankPower) + + " docIDs). "); + } + + this.slice = blockSlice; + this.jumpTable = jumpTable; + this.jumpTableEntryCount = jumpTableEntryCount; + this.denseRankPower = denseRankPower; + final int rankIndexShift = denseRankPower - 7; + this.denseRankTable = + denseRankPower == -1 ? null : new byte[DENSE_BLOCK_LONGS >> rankIndexShift]; + this.cost = cost; + } + + /** + * Helper method for using {@link #IndexedDISI(IndexInput, RandomAccessInput, int, byte, long)}. + * Creates a disiSlice for the IndexedDISI data blocks, without the jump-table. + * + * @param slice backing data, holding both blocks and jump-table. + * @param sliceDescription human readable slice designation. + * @param offset relative to the backing data. + * @param length full length of the IndexedDISI, including blocks and jump-table data. + * @param jumpTableEntryCount the number of blocks covered by the jump-table. + * @return a jumpTable containing the block jump-data or null if no such table exists. + * @throws IOException if a RandomAccessInput could not be created from slice. + */ + public static IndexInput createBlockSlice( + IndexInput slice, String sliceDescription, long offset, long length, int jumpTableEntryCount) + throws IOException { + long jumpTableBytes = jumpTableEntryCount < 0 ? 0 : jumpTableEntryCount * Integer.BYTES * 2; + return slice.slice(sliceDescription, offset, length - jumpTableBytes); + } + + /** + * Helper method for using {@link #IndexedDISI(IndexInput, RandomAccessInput, int, byte, long)}. + * Creates a RandomAccessInput covering only the jump-table data or null. + * + * @param slice backing data, holding both blocks and jump-table. + * @param offset relative to the backing data. + * @param length full length of the IndexedDISI, including blocks and jump-table data. + * @param jumpTableEntryCount the number of blocks covered by the jump-table. + * @return a jumpTable containing the block jump-data or null if no such table exists. + * @throws IOException if a RandomAccessInput could not be created from slice. + */ + public static RandomAccessInput createJumpTable( + IndexInput slice, long offset, long length, int jumpTableEntryCount) throws IOException { + if (jumpTableEntryCount <= 0) { + return null; + } else { + int jumpTableBytes = jumpTableEntryCount * Integer.BYTES * 2; + return slice.randomAccessSlice(offset + length - jumpTableBytes, jumpTableBytes); + } + } + + int block = -1; + long blockEnd; + long denseBitmapOffset = -1; // Only used for DENSE blocks + int nextBlockIndex = -1; + Method method; + + int doc = -1; + int index = -1; + + // SPARSE variables + boolean exists; + + // DENSE variables + long word; + int wordIndex = -1; + // number of one bits encountered so far, including those of `word` + int numberOfOnes; + // Used with rank for jumps inside of DENSE as they are absolute instead of relative + int denseOrigoIndex; + + // ALL variables + int gap; + + @Override + public int docID() { + return doc; + } + + @Override + public int advance(int target) throws IOException { + final int targetBlock = target & 0xFFFF0000; + if (block < targetBlock) { + advanceBlock(targetBlock); + } + if (block == targetBlock) { + if (method.advanceWithinBlock(this, target)) { + return doc; + } + readBlockHeader(); + } + boolean found = method.advanceWithinBlock(this, block); + assert found; + return doc; + } + + public boolean advanceExact(int target) throws IOException { + final int targetBlock = target & 0xFFFF0000; + if (block < targetBlock) { + advanceBlock(targetBlock); + } + boolean found = block == targetBlock && method.advanceExactWithinBlock(this, target); + this.doc = target; + return found; + } + + private void advanceBlock(int targetBlock) throws IOException { + final int blockIndex = targetBlock >> 16; + // If the destination block is 2 blocks or more ahead, we use the jump-table. + if (jumpTable != null && blockIndex >= (block >> 16) + 2) { + // If the jumpTableEntryCount is exceeded, there are no further bits. Last entry is always + // NO_MORE_DOCS + final int inRangeBlockIndex = + blockIndex < jumpTableEntryCount ? blockIndex : jumpTableEntryCount - 1; + final int index = jumpTable.readInt(inRangeBlockIndex * Integer.BYTES * 2); + final int offset = jumpTable.readInt(inRangeBlockIndex * Integer.BYTES * 2 + Integer.BYTES); + this.nextBlockIndex = index - 1; // -1 to compensate for the always-added 1 in readBlockHeader + slice.seek(offset); + readBlockHeader(); + return; + } + + // Fallback to iteration of blocks + do { + slice.seek(blockEnd); + readBlockHeader(); + } while (block < targetBlock); + } + + private void readBlockHeader() throws IOException { + block = Short.toUnsignedInt(slice.readShort()) << 16; + assert block >= 0; + final int numValues = 1 + Short.toUnsignedInt(slice.readShort()); + index = nextBlockIndex; + nextBlockIndex = index + numValues; + if (numValues <= MAX_ARRAY_LENGTH) { + method = Method.SPARSE; + blockEnd = slice.getFilePointer() + (numValues << 1); + } else if (numValues == 65536) { + method = Method.ALL; + blockEnd = slice.getFilePointer(); + gap = block - index - 1; + } else { + method = Method.DENSE; + denseBitmapOffset = + slice.getFilePointer() + (denseRankTable == null ? 0 : denseRankTable.length); + blockEnd = denseBitmapOffset + (1 << 13); + // Performance consideration: All rank (default 128 * 16 bits) are loaded up front. This + // should be fast with the + // reusable byte[] buffer, but it is still wasted if the DENSE block is iterated in small + // steps. + // If this results in too great a performance regression, a heuristic strategy might work + // where the rank data + // are loaded on first in-block advance, if said advance is > X docIDs. The hope being that a + // small first + // advance means that subsequent advances will be small too. + // Another alternative is to maintain an extra slice for DENSE rank, but IndexedDISI is + // already slice-heavy. + if (denseRankPower != -1) { + slice.readBytes(denseRankTable, 0, denseRankTable.length); + } + wordIndex = -1; + numberOfOnes = index + 1; + denseOrigoIndex = numberOfOnes; + } + } + + @Override + public int nextDoc() throws IOException { + return advance(doc + 1); + } + + public int index() { + return index; + } + + @Override + public long cost() { + return cost; + } + + enum Method { + SPARSE { + @Override + boolean advanceWithinBlock(IndexedDISI disi, int target) throws IOException { + final int targetInBlock = target & 0xFFFF; + // TODO: binary search + for (; disi.index < disi.nextBlockIndex; ) { + int doc = Short.toUnsignedInt(disi.slice.readShort()); + disi.index++; + if (doc >= targetInBlock) { + disi.doc = disi.block | doc; + disi.exists = true; + return true; + } + } + return false; + } + + @Override + boolean advanceExactWithinBlock(IndexedDISI disi, int target) throws IOException { + final int targetInBlock = target & 0xFFFF; + // TODO: binary search + if (target == disi.doc) { + return disi.exists; + } + for (; disi.index < disi.nextBlockIndex; ) { + int doc = Short.toUnsignedInt(disi.slice.readShort()); + disi.index++; + if (doc >= targetInBlock) { + if (doc != targetInBlock) { + disi.index--; + disi.slice.seek(disi.slice.getFilePointer() - Short.BYTES); + break; + } + disi.exists = true; + return true; + } + } + disi.exists = false; + return false; + } + }, + DENSE { + @Override + boolean advanceWithinBlock(IndexedDISI disi, int target) throws IOException { + final int targetInBlock = target & 0xFFFF; + final int targetWordIndex = targetInBlock >>> 6; + + // If possible, skip ahead using the rank cache + // If the distance between the current position and the target is < rank-longs + // there is no sense in using rank + if (disi.denseRankPower != -1 + && targetWordIndex - disi.wordIndex >= (1 << (disi.denseRankPower - 6))) { + rankSkip(disi, targetInBlock); + } + + for (int i = disi.wordIndex + 1; i <= targetWordIndex; ++i) { + disi.word = disi.slice.readLong(); + disi.numberOfOnes += Long.bitCount(disi.word); + } + disi.wordIndex = targetWordIndex; + + long leftBits = disi.word >>> target; + if (leftBits != 0L) { + disi.doc = target + Long.numberOfTrailingZeros(leftBits); + disi.index = disi.numberOfOnes - Long.bitCount(leftBits); + return true; + } + + // There were no set bits at the wanted position. Move forward until one is reached + while (++disi.wordIndex < 1024) { + // This could use the rank cache to skip empty spaces >= 512 bits, but it seems + // unrealistic + // that such blocks would be DENSE + disi.word = disi.slice.readLong(); + if (disi.word != 0) { + disi.index = disi.numberOfOnes; + disi.numberOfOnes += Long.bitCount(disi.word); + disi.doc = disi.block | (disi.wordIndex << 6) | Long.numberOfTrailingZeros(disi.word); + return true; + } + } + // No set bits in the block at or after the wanted position. + return false; + } + + @Override + boolean advanceExactWithinBlock(IndexedDISI disi, int target) throws IOException { + final int targetInBlock = target & 0xFFFF; + final int targetWordIndex = targetInBlock >>> 6; + + // If possible, skip ahead using the rank cache + // If the distance between the current position and the target is < rank-longs + // there is no sense in using rank + if (disi.denseRankPower != -1 + && targetWordIndex - disi.wordIndex >= (1 << (disi.denseRankPower - 6))) { + rankSkip(disi, targetInBlock); + } + + for (int i = disi.wordIndex + 1; i <= targetWordIndex; ++i) { + disi.word = disi.slice.readLong(); + disi.numberOfOnes += Long.bitCount(disi.word); + } + disi.wordIndex = targetWordIndex; + + long leftBits = disi.word >>> target; + disi.index = disi.numberOfOnes - Long.bitCount(leftBits); + return (leftBits & 1L) != 0; + } + }, + ALL { + @Override + boolean advanceWithinBlock(IndexedDISI disi, int target) { + disi.doc = target; + disi.index = target - disi.gap; + return true; + } + + @Override + boolean advanceExactWithinBlock(IndexedDISI disi, int target) { + disi.index = target - disi.gap; + return true; + } + }; + + /** + * Advance to the first doc from the block that is equal to or greater than {@code target}. + * Return true if there is such a doc and false otherwise. + */ + abstract boolean advanceWithinBlock(IndexedDISI disi, int target) throws IOException; + + /** + * Advance the iterator exactly to the position corresponding to the given {@code target} and + * return whether this document exists. + */ + abstract boolean advanceExactWithinBlock(IndexedDISI disi, int target) throws IOException; + } + + /** + * If the distance between the current position and the target is > 8 words, the rank cache will + * be used to guarantee a worst-case of 1 rank-lookup and 7 word-read-and-count-bits operations. + * Note: This does not guarantee a skip up to target, only up to nearest rank boundary. It is the + * responsibility of the caller to iterate further to reach target. + * + * @param disi standard DISI. + * @param targetInBlock lower 16 bits of the target + * @throws IOException if a DISI seek failed. + */ + private static void rankSkip(IndexedDISI disi, int targetInBlock) throws IOException { + assert disi.denseRankPower >= 0 : disi.denseRankPower; + // Resolve the rank as close to targetInBlock as possible (maximum distance is 8 longs) + // Note: rankOrigoOffset is tracked on block open, so it is absolute (e.g. don't add origo) + final int rankIndex = + targetInBlock >> disi.denseRankPower; // Default is 9 (8 longs: 2^3 * 2^6 = 512 docIDs) + + final int rank = + (disi.denseRankTable[rankIndex << 1] & 0xFF) << 8 + | (disi.denseRankTable[(rankIndex << 1) + 1] & 0xFF); + + // Position the counting logic just after the rank point + final int rankAlignedWordIndex = rankIndex << disi.denseRankPower >> 6; + disi.slice.seek(disi.denseBitmapOffset + rankAlignedWordIndex * Long.BYTES); + long rankWord = disi.slice.readLong(); + int denseNOO = rank + Long.bitCount(rankWord); + + disi.wordIndex = rankAlignedWordIndex; + disi.word = rankWord; + disi.numberOfOnes = disi.denseOrigoIndex + denseNOO; + } +} diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene80/Lucene80Codec.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene80/Lucene80Codec.java index 92b6a21ee63..7136276da04 100644 --- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene80/Lucene80Codec.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene80/Lucene80Codec.java @@ -35,7 +35,6 @@ import org.apache.lucene.codecs.StoredFieldsFormat; import org.apache.lucene.codecs.TermVectorsFormat; import org.apache.lucene.codecs.VectorFormat; import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat; -import org.apache.lucene.codecs.lucene80.Lucene80NormsFormat; import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat; import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; @@ -44,7 +43,7 @@ import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; * *

* - * @see org.apache.lucene.codecs.lucene80 package documentation for file format details. + * @see org.apache.lucene.backward_codecs.lucene80 package documentation for file format details. * @lucene.experimental */ public class Lucene80Codec extends Codec { diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene80/Lucene80DocValuesConsumer.java similarity index 98% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene80/Lucene80DocValuesConsumer.java index 118ba037a33..bfb03352140 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene80/Lucene80DocValuesConsumer.java @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene80; +package org.apache.lucene.backward_codecs.lucene80; -import static org.apache.lucene.codecs.lucene80.Lucene80DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT; -import static org.apache.lucene.codecs.lucene80.Lucene80DocValuesFormat.NUMERIC_BLOCK_SHIFT; -import static org.apache.lucene.codecs.lucene80.Lucene80DocValuesFormat.NUMERIC_BLOCK_SIZE; +import static org.apache.lucene.backward_codecs.lucene80.Lucene80DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT; +import static org.apache.lucene.backward_codecs.lucene80.Lucene80DocValuesFormat.NUMERIC_BLOCK_SHIFT; +import static org.apache.lucene.backward_codecs.lucene80.Lucene80DocValuesFormat.NUMERIC_BLOCK_SIZE; import java.io.Closeable; import java.io.IOException; @@ -403,7 +403,7 @@ final class Lucene80DocValuesConsumer extends DocValuesConsumer implements Close } class CompressedBinaryBlockWriter implements Closeable { - final FastCompressionHashTable ht = new LZ4.FastCompressionHashTable(); + final FastCompressionHashTable ht = new FastCompressionHashTable(); int uncompressedBlockLength = 0; int maxUncompressedBlockLength = 0; int numDocsInCurrentBlock = 0; @@ -772,10 +772,10 @@ final class Lucene80DocValuesConsumer extends DocValuesConsumer implements Close int maxLength = 0, maxBlockLength = 0; TermsEnum iterator = values.termsEnum(); - LZ4.FastCompressionHashTable ht = null; + FastCompressionHashTable ht = null; ByteArrayDataOutput bufferedOutput = null; if (compress) { - ht = new LZ4.FastCompressionHashTable(); + ht = new FastCompressionHashTable(); bufferedOutput = new ByteArrayDataOutput(termsDictBuffer); } @@ -840,7 +840,7 @@ final class Lucene80DocValuesConsumer extends DocValuesConsumer implements Close } private int compressAndGetTermsDictBlockLength( - ByteArrayDataOutput bufferedOutput, LZ4.FastCompressionHashTable ht) throws IOException { + ByteArrayDataOutput bufferedOutput, FastCompressionHashTable ht) throws IOException { int uncompressedLength = bufferedOutput.getPosition(); data.writeVInt(uncompressedLength); long before = data.getFilePointer(); diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene80/Lucene80DocValuesFormat.java similarity index 97% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesFormat.java rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene80/Lucene80DocValuesFormat.java index c8d59cc2870..4f80698cc63 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesFormat.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene80/Lucene80DocValuesFormat.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene80; +package org.apache.lucene.backward_codecs.lucene80; import java.io.IOException; import java.util.Objects; @@ -163,6 +163,11 @@ public final class Lucene80DocValuesFormat extends DocValuesFormat { this.mode = Objects.requireNonNull(mode); } + /** + * Note: although this format is only used on older versions, we need to keep the write logic in + * addition to the read logic. It's possible for doc values on older segments to be written to + * through doc values updates. + */ @Override public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException { return new Lucene80DocValuesConsumer( diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesProducer.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene80/Lucene80DocValuesProducer.java similarity index 99% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesProducer.java rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene80/Lucene80DocValuesProducer.java index 5813435d16a..67e69ee93d2 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesProducer.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene80/Lucene80DocValuesProducer.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene80; +package org.apache.lucene.backward_codecs.lucene80; import java.io.Closeable; import java.io.IOException; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80NormsFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene80/Lucene80NormsFormat.java similarity index 91% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80NormsFormat.java rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene80/Lucene80NormsFormat.java index 18205817eb9..28b191c2d6b 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80NormsFormat.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene80/Lucene80NormsFormat.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene80; +package org.apache.lucene.backward_codecs.lucene80; import java.io.IOException; import org.apache.lucene.codecs.CodecUtil; @@ -87,8 +87,7 @@ public class Lucene80NormsFormat extends NormsFormat { @Override public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException { - return new Lucene80NormsConsumer( - state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION); + throw new UnsupportedOperationException("Old codecs may only be used for reading"); } @Override @@ -97,10 +96,10 @@ public class Lucene80NormsFormat extends NormsFormat { state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION); } - private static final String DATA_CODEC = "Lucene80NormsData"; - private static final String DATA_EXTENSION = "nvd"; - private static final String METADATA_CODEC = "Lucene80NormsMetadata"; - private static final String METADATA_EXTENSION = "nvm"; + static final String DATA_CODEC = "Lucene80NormsData"; + static final String DATA_EXTENSION = "nvd"; + static final String METADATA_CODEC = "Lucene80NormsMetadata"; + static final String METADATA_EXTENSION = "nvm"; static final int VERSION_START = 0; static final int VERSION_CURRENT = VERSION_START; } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80NormsProducer.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene80/Lucene80NormsProducer.java similarity index 98% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80NormsProducer.java rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene80/Lucene80NormsProducer.java index ab26cda0edd..2fcd594c7f6 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80NormsProducer.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene80/Lucene80NormsProducer.java @@ -14,10 +14,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene80; +package org.apache.lucene.backward_codecs.lucene80; -import static org.apache.lucene.codecs.lucene80.Lucene80NormsFormat.VERSION_CURRENT; -import static org.apache.lucene.codecs.lucene80.Lucene80NormsFormat.VERSION_START; +import static org.apache.lucene.backward_codecs.lucene80.Lucene80NormsFormat.VERSION_CURRENT; +import static org.apache.lucene.backward_codecs.lucene80.Lucene80NormsFormat.VERSION_START; import java.io.IOException; import java.util.HashMap; diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene80/package-info.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene80/package-info.java index 2f8459ca19f..ade526b926a 100644 --- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene80/package-info.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene80/package-info.java @@ -15,5 +15,5 @@ * limitations under the License. */ -/** Lucene 8.0 file format. */ +/** Components from the Lucene 8.0 index format. */ package org.apache.lucene.backward_codecs.lucene80; diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene84/Lucene84Codec.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene84/Lucene84Codec.java index 4f9db5587f7..81125f56075 100644 --- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene84/Lucene84Codec.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene84/Lucene84Codec.java @@ -24,6 +24,7 @@ import org.apache.lucene.backward_codecs.lucene50.Lucene50StoredFieldsFormat.Mod import org.apache.lucene.backward_codecs.lucene60.Lucene60FieldInfosFormat; import org.apache.lucene.backward_codecs.lucene60.Lucene60PointsFormat; import org.apache.lucene.backward_codecs.lucene70.Lucene70SegmentInfoFormat; +import org.apache.lucene.backward_codecs.lucene80.Lucene80NormsFormat; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.CompoundFormat; import org.apache.lucene.codecs.DocValuesFormat; @@ -38,7 +39,6 @@ import org.apache.lucene.codecs.StoredFieldsFormat; import org.apache.lucene.codecs.TermVectorsFormat; import org.apache.lucene.codecs.VectorFormat; import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat; -import org.apache.lucene.codecs.lucene80.Lucene80NormsFormat; import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat; import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; @@ -173,7 +173,7 @@ public class Lucene84Codec extends Codec { private final NormsFormat normsFormat = new Lucene80NormsFormat(); @Override - public final NormsFormat normsFormat() { + public NormsFormat normsFormat() { return normsFormat; } } diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene86/Lucene86Codec.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene86/Lucene86Codec.java index 817643140fe..78fb85aec10 100644 --- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene86/Lucene86Codec.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene86/Lucene86Codec.java @@ -22,6 +22,7 @@ import org.apache.lucene.backward_codecs.lucene50.Lucene50CompoundFormat; import org.apache.lucene.backward_codecs.lucene50.Lucene50LiveDocsFormat; import org.apache.lucene.backward_codecs.lucene50.Lucene50StoredFieldsFormat; import org.apache.lucene.backward_codecs.lucene60.Lucene60FieldInfosFormat; +import org.apache.lucene.backward_codecs.lucene80.Lucene80NormsFormat; import org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.CompoundFormat; @@ -37,7 +38,6 @@ import org.apache.lucene.codecs.StoredFieldsFormat; import org.apache.lucene.codecs.TermVectorsFormat; import org.apache.lucene.codecs.VectorFormat; import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat; -import org.apache.lucene.codecs.lucene80.Lucene80NormsFormat; import org.apache.lucene.codecs.lucene86.Lucene86PointsFormat; import org.apache.lucene.codecs.lucene86.Lucene86SegmentInfoFormat; import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat; @@ -49,8 +49,8 @@ import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; * *

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 + * @see org.apache.lucene.codecs.lucene86 package documentation for file format details. */ public class Lucene86Codec extends Codec { private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat(); @@ -175,7 +175,7 @@ public class Lucene86Codec extends Codec { private final NormsFormat normsFormat = new Lucene80NormsFormat(); @Override - public final NormsFormat normsFormat() { + public NormsFormat normsFormat() { return normsFormat; } } diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene87/Lucene87Codec.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene87/Lucene87Codec.java index 373a7d384e5..181e8fb912e 100644 --- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene87/Lucene87Codec.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene87/Lucene87Codec.java @@ -21,6 +21,8 @@ import java.util.Objects; import org.apache.lucene.backward_codecs.lucene50.Lucene50CompoundFormat; import org.apache.lucene.backward_codecs.lucene50.Lucene50LiveDocsFormat; import org.apache.lucene.backward_codecs.lucene60.Lucene60FieldInfosFormat; +import org.apache.lucene.backward_codecs.lucene80.Lucene80DocValuesFormat; +import org.apache.lucene.backward_codecs.lucene80.Lucene80NormsFormat; import org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.CompoundFormat; @@ -36,8 +38,6 @@ import org.apache.lucene.codecs.StoredFieldsFormat; import org.apache.lucene.codecs.TermVectorsFormat; import org.apache.lucene.codecs.VectorFormat; import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat; -import org.apache.lucene.codecs.lucene80.Lucene80DocValuesFormat; -import org.apache.lucene.codecs.lucene80.Lucene80NormsFormat; import org.apache.lucene.codecs.lucene86.Lucene86PointsFormat; import org.apache.lucene.codecs.lucene86.Lucene86SegmentInfoFormat; import org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat; @@ -187,7 +187,7 @@ public class Lucene87Codec extends Codec { private final NormsFormat normsFormat = new Lucene80NormsFormat(); @Override - public final NormsFormat normsFormat() { + public NormsFormat normsFormat() { return normsFormat; } } diff --git a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat index 8f034f83e80..f058ccd9ffb 100644 --- a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat +++ b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat @@ -14,3 +14,4 @@ # limitations under the License. org.apache.lucene.backward_codecs.lucene70.Lucene70DocValuesFormat +org.apache.lucene.backward_codecs.lucene80.Lucene80DocValuesFormat diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/Lucene87/Lucene87RWCodec.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/Lucene87/Lucene87RWCodec.java index 214085c86a1..8fac2d6076b 100644 --- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/Lucene87/Lucene87RWCodec.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/Lucene87/Lucene87RWCodec.java @@ -14,12 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.backward_codecs.Lucene87; +package org.apache.lucene.backward_codecs.lucene87; import org.apache.lucene.backward_codecs.lucene50.Lucene50RWCompoundFormat; +import org.apache.lucene.backward_codecs.lucene80.Lucene80RWNormsFormat; import org.apache.lucene.backward_codecs.lucene84.Lucene84RWPostingsFormat; -import org.apache.lucene.backward_codecs.lucene87.Lucene87Codec; import org.apache.lucene.codecs.CompoundFormat; +import org.apache.lucene.codecs.NormsFormat; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; @@ -40,6 +41,11 @@ public class Lucene87RWCodec extends Lucene87Codec { return new Lucene50RWCompoundFormat(); } + @Override + public NormsFormat normsFormat() { + return new Lucene80RWNormsFormat(); + } + @Override public PostingsFormat postingsFormat() { return postingsFormat; diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene50/TestLucene50CompoundFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene50/TestLucene50CompoundFormat.java index 28624bf8b02..de9f72cfec2 100644 --- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene50/TestLucene50CompoundFormat.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene50/TestLucene50CompoundFormat.java @@ -16,7 +16,7 @@ */ package org.apache.lucene.backward_codecs.lucene50; -import org.apache.lucene.backward_codecs.Lucene87.Lucene87RWCodec; +import org.apache.lucene.backward_codecs.lucene87.Lucene87RWCodec; import org.apache.lucene.codecs.Codec; import org.apache.lucene.index.BaseCompoundFormatTestCase; diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene80/BaseLucene80DocValuesFormatTestCase.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/BaseLucene80DocValuesFormatTestCase.java similarity index 99% rename from lucene/core/src/test/org/apache/lucene/codecs/lucene80/BaseLucene80DocValuesFormatTestCase.java rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/BaseLucene80DocValuesFormatTestCase.java index f1f951824bd..fce0a390ea6 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene80/BaseLucene80DocValuesFormatTestCase.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/BaseLucene80DocValuesFormatTestCase.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene80; +package org.apache.lucene.backward_codecs.lucene80; import java.io.IOException; import java.util.ArrayList; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80NormsConsumer.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/Lucene80NormsConsumer.java similarity index 97% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80NormsConsumer.java rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/Lucene80NormsConsumer.java index 72a898b8983..ea5c4df34a1 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80NormsConsumer.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/Lucene80NormsConsumer.java @@ -14,9 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene80; +package org.apache.lucene.backward_codecs.lucene80; -import static org.apache.lucene.codecs.lucene80.Lucene80NormsFormat.VERSION_CURRENT; +import static org.apache.lucene.backward_codecs.lucene80.Lucene80NormsFormat.VERSION_CURRENT; import java.io.IOException; import org.apache.lucene.codecs.CodecUtil; diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/Lucene80RWNormsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/Lucene80RWNormsFormat.java new file mode 100644 index 00000000000..1eee0f256f9 --- /dev/null +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/Lucene80RWNormsFormat.java @@ -0,0 +1,30 @@ +/* + * 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.lucene80; + +import java.io.IOException; +import org.apache.lucene.codecs.NormsConsumer; +import org.apache.lucene.index.SegmentWriteState; + +public class Lucene80RWNormsFormat extends Lucene80NormsFormat { + + @Override + public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException { + return new Lucene80NormsConsumer( + state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION); + } +} diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestBestCompressionLucene80DocValuesFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/TestBestCompressionLucene80DocValuesFormat.java similarity index 96% rename from lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestBestCompressionLucene80DocValuesFormat.java rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/TestBestCompressionLucene80DocValuesFormat.java index 1fafa551334..78bb84850dd 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestBestCompressionLucene80DocValuesFormat.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/TestBestCompressionLucene80DocValuesFormat.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene80; +package org.apache.lucene.backward_codecs.lucene80; import org.apache.lucene.codecs.Codec; import org.apache.lucene.util.TestUtil; diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestBestSpeedLucene80DocValuesFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/TestBestSpeedLucene80DocValuesFormat.java similarity index 95% rename from lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestBestSpeedLucene80DocValuesFormat.java rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/TestBestSpeedLucene80DocValuesFormat.java index 3027842c646..283c42f1c1a 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestBestSpeedLucene80DocValuesFormat.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/TestBestSpeedLucene80DocValuesFormat.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene80; +package org.apache.lucene.backward_codecs.lucene80; import org.apache.lucene.codecs.Codec; import org.apache.lucene.util.TestUtil; diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestDocValuesCompression.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/TestDocValuesCompression.java similarity index 97% rename from lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestDocValuesCompression.java rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/TestDocValuesCompression.java index fb5b66a3ecb..1d601640777 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestDocValuesCompression.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/TestDocValuesCompression.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene80; +package org.apache.lucene.backward_codecs.lucene80; import java.io.IOException; import java.util.ArrayList; @@ -25,7 +25,6 @@ import java.util.Set; import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.standard.StandardAnalyzer; import org.apache.lucene.codecs.Codec; -import org.apache.lucene.codecs.lucene90.Lucene90Codec; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.document.SortedDocValuesField; @@ -43,8 +42,13 @@ import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.TestUtil; public class TestDocValuesCompression extends LuceneTestCase { - private final Codec bestSpeed = new Lucene90Codec(Lucene90Codec.Mode.BEST_SPEED); - private final Codec bestCompression = new Lucene90Codec(Lucene90Codec.Mode.BEST_COMPRESSION); + private final Codec bestSpeed = + TestUtil.alwaysDocValuesFormat( + new Lucene80DocValuesFormat(Lucene80DocValuesFormat.Mode.BEST_SPEED)); + + private final Codec bestCompression = + TestUtil.alwaysDocValuesFormat( + new Lucene80DocValuesFormat(Lucene80DocValuesFormat.Mode.BEST_COMPRESSION)); public void testTermsDictCompressionForLowCardinalityFields() throws IOException { final int CARDINALITY = Lucene80DocValuesFormat.TERMS_DICT_BLOCK_COMPRESSION_THRESHOLD - 1; diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/TestIndexedDISI.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/TestIndexedDISI.java new file mode 100644 index 00000000000..bf54bf317c9 --- /dev/null +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/TestIndexedDISI.java @@ -0,0 +1,593 @@ +/* + * 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.lucene80; + +import java.io.IOException; +import java.util.Random; +import org.apache.lucene.search.DocIdSetIterator; +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.store.RandomAccessInput; +import org.apache.lucene.util.BitSet; +import org.apache.lucene.util.BitSetIterator; +import org.apache.lucene.util.FixedBitSet; +import org.apache.lucene.util.LuceneTestCase; +import org.apache.lucene.util.SparseFixedBitSet; +import org.apache.lucene.util.TestUtil; + +public class TestIndexedDISI extends LuceneTestCase { + + public void testEmpty() throws IOException { + int maxDoc = TestUtil.nextInt(random(), 1, 100000); + BitSet set = new SparseFixedBitSet(maxDoc); + try (Directory dir = newDirectory()) { + doTest(set, dir); + } + } + + // EMPTY blocks are special with regard to jumps as they have size 0 + @Nightly + public void testEmptyBlocks() throws IOException { + final int B = 65536; + int maxDoc = B * 11; + BitSet set = new SparseFixedBitSet(maxDoc); + // block 0: EMPTY + set.set(B + 5); // block 1: SPARSE + // block 2: EMPTY + // block 3: EMPTY + set.set(B * 4 + 5); // block 4: SPARSE + + for (int i = 0; i < B; i++) { + set.set(B * 6 + i); // block 6: ALL + } + for (int i = 0; i < B; i += 3) { + set.set(B * 7 + i); // block 7: DENSE + } + for (int i = 0; i < B; i++) { + if (i != 32768) { + set.set(B * 8 + i); // block 8: DENSE (all-1) + } + } + // block 9-11: EMPTY + + try (Directory dir = newDirectory()) { + doTestAllSingleJump(set, dir); + } + + // Change the first block to DENSE to see if jump-tables sets to position 0 + set.set(0); + try (Directory dir = newDirectory()) { + doTestAllSingleJump(set, dir); + } + } + + // EMPTY blocks are special with regard to jumps as they have size 0 + public void testLastEmptyBlocks() throws IOException { + final int B = 65536; + int maxDoc = B * 3; + BitSet set = new SparseFixedBitSet(maxDoc); + for (int docID = 0; docID < B * 2; docID++) { // first 2 blocks are ALL + set.set(docID); + } + // Last block is EMPTY + + try (Directory dir = newDirectory()) { + doTestAllSingleJump(set, dir); + assertAdvanceBeyondEnd(set, dir); + } + } + + // Checks that advance after the end of the blocks has been reached has the correct behaviour + private void assertAdvanceBeyondEnd(BitSet set, Directory dir) throws IOException { + final int cardinality = set.cardinality(); + final byte denseRankPower = 9; // Not tested here so fixed to isolate factors + long length; + int jumpTableentryCount; + try (IndexOutput out = dir.createOutput("bar", IOContext.DEFAULT)) { + jumpTableentryCount = + IndexedDISI.writeBitSet(new BitSetIterator(set, cardinality), out, denseRankPower); + } + + try (IndexInput in = dir.openInput("bar", IOContext.DEFAULT)) { + BitSetIterator disi2 = new BitSetIterator(set, cardinality); + int doc = disi2.docID(); + int index = 0; + while (doc < cardinality) { + doc = disi2.nextDoc(); + index++; + } + + IndexedDISI disi = + new IndexedDISI(in, 0L, in.length(), jumpTableentryCount, denseRankPower, cardinality); + // Advance 1 docID beyond end + assertFalse( + "There should be no set bit beyond the valid docID range", + disi.advanceExact(set.length())); + disi.advance( + doc); // Should be the special docID signifyin NO_MORE_DOCS from the BitSetIterator + assertEquals( + "The index when advancing beyond the last defined docID should be correct", + index, + disi.index() + 1); // disi.index()+1 as the while-loop also counts the NO_MORE_DOCS + } + } + + // TODO: can this be toned down? + @Nightly + public void testRandomBlocks() throws IOException { + final int BLOCKS = 5; + BitSet set = createSetWithRandomBlocks(BLOCKS); + try (Directory dir = newDirectory()) { + doTestAllSingleJump(set, dir); + } + } + + // When doing merges in Lucene80NormsProducer, IndexedDISI are created from slices where the + // offset is not 0 + public void testPositionNotZero() throws IOException { + final int BLOCKS = 10; + final byte denseRankPower = + rarely() ? -1 : (byte) (random().nextInt(7) + 7); // sane + chance of disable + + BitSet set = createSetWithRandomBlocks(BLOCKS); + try (Directory dir = newDirectory()) { + final int cardinality = set.cardinality(); + int jumpTableEntryCount; + try (IndexOutput out = dir.createOutput("foo", IOContext.DEFAULT)) { + jumpTableEntryCount = + IndexedDISI.writeBitSet(new BitSetIterator(set, cardinality), out, denseRankPower); + } + try (IndexInput fullInput = dir.openInput("foo", IOContext.DEFAULT)) { + IndexInput blockData = + IndexedDISI.createBlockSlice( + fullInput, "blocks", 0, fullInput.length(), jumpTableEntryCount); + blockData.seek(random().nextInt((int) blockData.length())); + + RandomAccessInput jumpTable = + IndexedDISI.createJumpTable(fullInput, 0, fullInput.length(), jumpTableEntryCount); + IndexedDISI disi = + new IndexedDISI(blockData, jumpTable, jumpTableEntryCount, denseRankPower, cardinality); + // This failed at some point during LUCENE-8585 development as it did not reset the slice + // position + disi.advanceExact(BLOCKS * 65536 - 1); + } + } + } + + private BitSet createSetWithRandomBlocks(int blockCount) { + final int B = 65536; + BitSet set = new SparseFixedBitSet(blockCount * B); + for (int block = 0; block < blockCount; block++) { + switch (random().nextInt(4)) { + case 0: + { // EMPTY + break; + } + case 1: + { // ALL + for (int docID = block * B; docID < (block + 1) * B; docID++) { + set.set(docID); + } + break; + } + case 2: + { // SPARSE ( < 4096 ) + for (int docID = block * B; docID < (block + 1) * B; docID += 101) { + set.set(docID); + } + break; + } + case 3: + { // DENSE ( >= 4096 ) + for (int docID = block * B; docID < (block + 1) * B; docID += 3) { + set.set(docID); + } + break; + } + default: + throw new IllegalStateException( + "Modulo logic error: there should only be 4 possibilities"); + } + } + return set; + } + + private void doTestAllSingleJump(BitSet set, Directory dir) throws IOException { + final int cardinality = set.cardinality(); + final byte denseRankPower = + rarely() ? -1 : (byte) (random().nextInt(7) + 7); // sane + chance of disable + long length; + int jumpTableentryCount; + try (IndexOutput out = dir.createOutput("foo", IOContext.DEFAULT)) { + jumpTableentryCount = + IndexedDISI.writeBitSet(new BitSetIterator(set, cardinality), out, denseRankPower); + length = out.getFilePointer(); + } + + try (IndexInput in = dir.openInput("foo", IOContext.DEFAULT)) { + for (int i = 0; i < set.length(); i++) { + IndexedDISI disi = + new IndexedDISI(in, 0L, length, jumpTableentryCount, denseRankPower, cardinality); + assertEquals( + "The bit at " + i + " should be correct with advanceExact", + set.get(i), + disi.advanceExact(i)); + + IndexedDISI disi2 = + new IndexedDISI(in, 0L, length, jumpTableentryCount, denseRankPower, cardinality); + disi2.advance(i); + // Proper sanity check with jump tables as an error could make them seek backwards + assertTrue( + "The docID should at least be " + + i + + " after advance(" + + i + + ") but was " + + disi2.docID(), + i <= disi2.docID()); + if (set.get(i)) { + assertEquals("The docID should be present with advance", i, disi2.docID()); + } else { + assertNotSame("The docID should not be present with advance", i, disi2.docID()); + } + } + } + } + + public void testOneDoc() throws IOException { + int maxDoc = TestUtil.nextInt(random(), 1, 100000); + BitSet set = new SparseFixedBitSet(maxDoc); + set.set(random().nextInt(maxDoc)); + try (Directory dir = newDirectory()) { + doTest(set, dir); + } + } + + public void testTwoDocs() throws IOException { + int maxDoc = TestUtil.nextInt(random(), 1, 100000); + BitSet set = new SparseFixedBitSet(maxDoc); + set.set(random().nextInt(maxDoc)); + set.set(random().nextInt(maxDoc)); + try (Directory dir = newDirectory()) { + doTest(set, dir); + } + } + + public void testAllDocs() throws IOException { + int maxDoc = TestUtil.nextInt(random(), 1, 100000); + FixedBitSet set = new FixedBitSet(maxDoc); + set.set(1, maxDoc); + try (Directory dir = newDirectory()) { + doTest(set, dir); + } + } + + public void testHalfFull() throws IOException { + int maxDoc = TestUtil.nextInt(random(), 1, 100000); + BitSet set = new SparseFixedBitSet(maxDoc); + for (int i = random().nextInt(2); i < maxDoc; i += TestUtil.nextInt(random(), 1, 3)) { + set.set(i); + } + try (Directory dir = newDirectory()) { + doTest(set, dir); + } + } + + public void testDocRange() throws IOException { + try (Directory dir = newDirectory()) { + for (int iter = 0; iter < 10; ++iter) { + int maxDoc = TestUtil.nextInt(random(), 1, 1000000); + FixedBitSet set = new FixedBitSet(maxDoc); + final int start = random().nextInt(maxDoc); + final int end = TestUtil.nextInt(random(), start + 1, maxDoc); + set.set(start, end); + doTest(set, dir); + } + } + } + + public void testSparseDenseBoundary() throws IOException { + try (Directory dir = newDirectory()) { + FixedBitSet set = new FixedBitSet(200000); + int start = 65536 + random().nextInt(100); + final byte denseRankPower = + rarely() ? -1 : (byte) (random().nextInt(7) + 7); // sane + chance of disable + + // we set MAX_ARRAY_LENGTH bits so the encoding will be sparse + set.set(start, start + IndexedDISI.MAX_ARRAY_LENGTH); + long length; + int jumpTableEntryCount; + try (IndexOutput out = dir.createOutput("sparse", IOContext.DEFAULT)) { + jumpTableEntryCount = + IndexedDISI.writeBitSet( + new BitSetIterator(set, IndexedDISI.MAX_ARRAY_LENGTH), out, denseRankPower); + length = out.getFilePointer(); + } + try (IndexInput in = dir.openInput("sparse", IOContext.DEFAULT)) { + IndexedDISI disi = + new IndexedDISI( + in, 0L, length, jumpTableEntryCount, denseRankPower, IndexedDISI.MAX_ARRAY_LENGTH); + assertEquals(start, disi.nextDoc()); + assertEquals(IndexedDISI.Method.SPARSE, disi.method); + } + doTest(set, dir); + + // now we set one more bit so the encoding will be dense + set.set(start + IndexedDISI.MAX_ARRAY_LENGTH + random().nextInt(100)); + try (IndexOutput out = dir.createOutput("bar", IOContext.DEFAULT)) { + IndexedDISI.writeBitSet( + new BitSetIterator(set, IndexedDISI.MAX_ARRAY_LENGTH + 1), out, denseRankPower); + length = out.getFilePointer(); + } + try (IndexInput in = dir.openInput("bar", IOContext.DEFAULT)) { + IndexedDISI disi = + new IndexedDISI( + in, + 0L, + length, + jumpTableEntryCount, + denseRankPower, + IndexedDISI.MAX_ARRAY_LENGTH + 1); + assertEquals(start, disi.nextDoc()); + assertEquals(IndexedDISI.Method.DENSE, disi.method); + } + doTest(set, dir); + } + } + + public void testOneDocMissing() throws IOException { + int maxDoc = TestUtil.nextInt(random(), 1, 1000000); + FixedBitSet set = new FixedBitSet(maxDoc); + set.set(0, maxDoc); + set.clear(random().nextInt(maxDoc)); + try (Directory dir = newDirectory()) { + doTest(set, dir); + } + } + + public void testFewMissingDocs() throws IOException { + try (Directory dir = newDirectory()) { + int numIters = atLeast(10); + for (int iter = 0; iter < numIters; ++iter) { + int maxDoc = TestUtil.nextInt(random(), 1, 100000); + FixedBitSet set = new FixedBitSet(maxDoc); + set.set(0, maxDoc); + final int numMissingDocs = TestUtil.nextInt(random(), 2, 1000); + for (int i = 0; i < numMissingDocs; ++i) { + set.clear(random().nextInt(maxDoc)); + } + doTest(set, dir); + } + } + } + + public void testDenseMultiBlock() throws IOException { + try (Directory dir = newDirectory()) { + int maxDoc = 10 * 65536; // 10 blocks + FixedBitSet set = new FixedBitSet(maxDoc); + for (int i = 0; i < maxDoc; i += 2) { // Set every other to ensure dense + set.set(i); + } + doTest(set, dir); + } + } + + public void testIllegalDenseRankPower() throws IOException { + + // Legal values + for (byte denseRankPower : new byte[] {-1, 7, 8, 9, 10, 11, 12, 13, 14, 15}) { + createAndOpenDISI(denseRankPower, denseRankPower); + } + + // Illegal values + for (byte denseRankPower : new byte[] {-2, 0, 1, 6, 16}) { + try { + createAndOpenDISI( + denseRankPower, (byte) 8); // Illegal write, legal read (should not reach read) + fail( + "Trying to create an IndexedDISI data stream with denseRankPower-read " + + denseRankPower + + " and denseRankPower-write 8 should fail"); + } catch (IllegalArgumentException e) { + // Expected + } + try { + createAndOpenDISI( + (byte) 8, denseRankPower); // Legal write, illegal read (should reach read) + fail( + "Trying to create an IndexedDISI data stream with denseRankPower-write 8 and denseRankPower-read " + + denseRankPower + + " should fail"); + } catch (IllegalArgumentException e) { + // Expected + } + } + } + + private void createAndOpenDISI(byte denseRankPowerWrite, byte denseRankPowerRead) + throws IOException { + BitSet set = new FixedBitSet(10); + set.set(set.length() - 1); + try (Directory dir = newDirectory()) { + long length; + int jumpTableEntryCount = -1; + try (IndexOutput out = dir.createOutput("foo", IOContext.DEFAULT)) { + jumpTableEntryCount = + IndexedDISI.writeBitSet( + new BitSetIterator(set, set.cardinality()), out, denseRankPowerWrite); + length = out.getFilePointer(); + } + try (IndexInput in = dir.openInput("foo", IOContext.DEFAULT)) { + IndexedDISI disi = + new IndexedDISI( + in, 0L, length, jumpTableEntryCount, denseRankPowerRead, set.cardinality()); + } + // This tests the legality of the denseRankPower only, so we don't do anything with the disi + } + } + + public void testOneDocMissingFixed() throws IOException { + int maxDoc = 9699; + final byte denseRankPower = + rarely() ? -1 : (byte) (random().nextInt(7) + 7); // sane + chance of disable + FixedBitSet set = new FixedBitSet(maxDoc); + set.set(0, maxDoc); + set.clear(1345); + try (Directory dir = newDirectory()) { + + final int cardinality = set.cardinality(); + long length; + int jumpTableentryCount; + try (IndexOutput out = dir.createOutput("foo", IOContext.DEFAULT)) { + jumpTableentryCount = + IndexedDISI.writeBitSet(new BitSetIterator(set, cardinality), out, denseRankPower); + length = out.getFilePointer(); + } + + int step = 16000; + try (IndexInput in = dir.openInput("foo", IOContext.DEFAULT)) { + IndexedDISI disi = + new IndexedDISI(in, 0L, length, jumpTableentryCount, denseRankPower, cardinality); + BitSetIterator disi2 = new BitSetIterator(set, cardinality); + assertAdvanceEquality(disi, disi2, step); + } + } + } + + public void testRandom() throws IOException { + try (Directory dir = newDirectory()) { + int numIters = atLeast(3); + for (int i = 0; i < numIters; ++i) { + doTestRandom(dir); + } + } + } + + private void doTestRandom(Directory dir) throws IOException { + Random random = random(); + final int maxStep = TestUtil.nextInt(random, 1, 1 << TestUtil.nextInt(random, 2, 20)); + final int numDocs = TestUtil.nextInt(random, 1, Math.min(100000, Integer.MAX_VALUE / maxStep)); + BitSet docs = new SparseFixedBitSet(numDocs * (maxStep + 1)); + int lastDoc = -1; + for (int doc = -1, i = 0; i < numDocs; ++i) { + doc += TestUtil.nextInt(random, 1, maxStep); + docs.set(doc); + lastDoc = doc; + } + final int maxDoc = lastDoc + TestUtil.nextInt(random, 1, 100); + + BitSet set = BitSet.of(new BitSetIterator(docs, docs.approximateCardinality()), maxDoc); + doTest(set, dir); + } + + private void doTest(BitSet set, Directory dir) throws IOException { + final int cardinality = set.cardinality(); + final byte denseRankPower = + rarely() ? -1 : (byte) (random().nextInt(7) + 7); // sane + chance of disable + long length; + int jumpTableentryCount; + try (IndexOutput out = dir.createOutput("foo", IOContext.DEFAULT)) { + jumpTableentryCount = + IndexedDISI.writeBitSet(new BitSetIterator(set, cardinality), out, denseRankPower); + length = out.getFilePointer(); + } + + try (IndexInput in = dir.openInput("foo", IOContext.DEFAULT)) { + IndexedDISI disi = + new IndexedDISI(in, 0L, length, jumpTableentryCount, denseRankPower, cardinality); + BitSetIterator disi2 = new BitSetIterator(set, cardinality); + assertSingleStepEquality(disi, disi2); + } + + for (int step : new int[] {1, 10, 100, 1000, 10000, 100000}) { + try (IndexInput in = dir.openInput("foo", IOContext.DEFAULT)) { + IndexedDISI disi = + new IndexedDISI(in, 0L, length, jumpTableentryCount, denseRankPower, cardinality); + BitSetIterator disi2 = new BitSetIterator(set, cardinality); + assertAdvanceEquality(disi, disi2, step); + } + } + + for (int step : new int[] {10, 100, 1000, 10000, 100000}) { + try (IndexInput in = dir.openInput("foo", IOContext.DEFAULT)) { + IndexedDISI disi = + new IndexedDISI(in, 0L, length, jumpTableentryCount, denseRankPower, cardinality); + BitSetIterator disi2 = new BitSetIterator(set, cardinality); + int disi2length = set.length(); + assertAdvanceExactRandomized(disi, disi2, disi2length, step); + } + } + + dir.deleteFile("foo"); + } + + private void assertAdvanceExactRandomized( + IndexedDISI disi, BitSetIterator disi2, int disi2length, int step) throws IOException { + int index = -1; + Random random = random(); + for (int target = 0; target < disi2length; ) { + target += TestUtil.nextInt(random, 0, step); + int doc = disi2.docID(); + while (doc < target) { + doc = disi2.nextDoc(); + index++; + } + + boolean exists = disi.advanceExact(target); + assertEquals(doc == target, exists); + if (exists) { + assertEquals(index, disi.index()); + } else if (random.nextBoolean()) { + assertEquals(doc, disi.nextDoc()); + // This is a bit strange when doc == NO_MORE_DOCS as the index overcounts in the disi2 + // while-loop + assertEquals(index, disi.index()); + target = doc; + } + } + } + + private void assertSingleStepEquality(IndexedDISI disi, BitSetIterator disi2) throws IOException { + int i = 0; + for (int doc = disi2.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = disi2.nextDoc()) { + assertEquals(doc, disi.nextDoc()); + assertEquals(i++, disi.index()); + } + assertEquals(DocIdSetIterator.NO_MORE_DOCS, disi.nextDoc()); + } + + private void assertAdvanceEquality(IndexedDISI disi, BitSetIterator disi2, int step) + throws IOException { + int index = -1; + while (true) { + int target = disi2.docID() + step; + int doc; + do { + doc = disi2.nextDoc(); + index++; + } while (doc < target); + assertEquals(doc, disi.advance(target)); + if (doc == DocIdSetIterator.NO_MORE_DOCS) { + break; + } + assertEquals( + "Expected equality using step " + step + " at docID " + doc, index, disi.index()); + } + } +} diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestLucene80NormsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/TestLucene80NormsFormat.java similarity index 86% rename from lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestLucene80NormsFormat.java rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/TestLucene80NormsFormat.java index 36047f06797..f46037f087c 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestLucene80NormsFormat.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/TestLucene80NormsFormat.java @@ -14,15 +14,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene80; +package org.apache.lucene.backward_codecs.lucene80; +import org.apache.lucene.backward_codecs.lucene87.Lucene87RWCodec; import org.apache.lucene.codecs.Codec; import org.apache.lucene.index.BaseNormsFormatTestCase; -import org.apache.lucene.util.TestUtil; /** Tests Lucene80NormsFormat */ public class TestLucene80NormsFormat extends BaseNormsFormatTestCase { - private final Codec codec = TestUtil.getDefaultCodec(); + private final Codec codec = new Lucene87RWCodec(); @Override protected Codec getCodec() { diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestLucene80NormsFormatMergeInstance.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/TestLucene80NormsFormatMergeInstance.java similarity index 95% rename from lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestLucene80NormsFormatMergeInstance.java rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/TestLucene80NormsFormatMergeInstance.java index d01d049c518..eaa173804cb 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestLucene80NormsFormatMergeInstance.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/TestLucene80NormsFormatMergeInstance.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene80; +package org.apache.lucene.backward_codecs.lucene80; /** Test the merge instance of the Lucene80 norms format. */ public class TestLucene80NormsFormatMergeInstance extends TestLucene80NormsFormat { diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene84/Lucene84RWCodec.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene84/Lucene84RWCodec.java index a5e34c90fdc..3b70b2c0edd 100644 --- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene84/Lucene84RWCodec.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene84/Lucene84RWCodec.java @@ -20,7 +20,9 @@ import org.apache.lucene.backward_codecs.lucene50.Lucene50RWCompoundFormat; import org.apache.lucene.backward_codecs.lucene50.Lucene50RWStoredFieldsFormat; import org.apache.lucene.backward_codecs.lucene60.Lucene60RWPointsFormat; import org.apache.lucene.backward_codecs.lucene70.Lucene70RWSegmentInfoFormat; +import org.apache.lucene.backward_codecs.lucene80.Lucene80RWNormsFormat; import org.apache.lucene.codecs.CompoundFormat; +import org.apache.lucene.codecs.NormsFormat; import org.apache.lucene.codecs.PointsFormat; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.SegmentInfoFormat; @@ -54,6 +56,11 @@ public class Lucene84RWCodec extends Lucene84Codec { return new Lucene50RWStoredFieldsFormat(); } + @Override + public NormsFormat normsFormat() { + return new Lucene80RWNormsFormat(); + } + @Override public PostingsFormat postingsFormat() { return postingsFormat; diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene86/Lucene86RWCodec.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene86/Lucene86RWCodec.java index c8dd36b5dbc..97d56333187 100644 --- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene86/Lucene86RWCodec.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene86/Lucene86RWCodec.java @@ -19,8 +19,10 @@ package org.apache.lucene.backward_codecs.lucene86; import org.apache.lucene.backward_codecs.lucene50.Lucene50RWCompoundFormat; import org.apache.lucene.backward_codecs.lucene50.Lucene50RWStoredFieldsFormat; import org.apache.lucene.backward_codecs.lucene50.Lucene50StoredFieldsFormat; +import org.apache.lucene.backward_codecs.lucene80.Lucene80RWNormsFormat; 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.perfield.PerFieldPostingsFormat; @@ -53,6 +55,11 @@ public class Lucene86RWCodec extends Lucene86Codec { return storedFieldsFormat; } + @Override + public NormsFormat normsFormat() { + return new Lucene80RWNormsFormat(); + } + @Override public PostingsFormat postingsFormat() { return postingsFormat; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene80/IndexedDISI.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/IndexedDISI.java similarity index 99% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene80/IndexedDISI.java rename to lucene/core/src/java/org/apache/lucene/codecs/lucene90/IndexedDISI.java index 472dad123ec..22da100fb47 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene80/IndexedDISI.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/IndexedDISI.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene80; +package org.apache.lucene.codecs.lucene90; import java.io.DataInput; import java.io.IOException; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90Codec.java index 65c866de218..770f3528522 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90Codec.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90Codec.java @@ -31,8 +31,6 @@ import org.apache.lucene.codecs.StoredFieldsFormat; import org.apache.lucene.codecs.TermVectorsFormat; import org.apache.lucene.codecs.VectorFormat; import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat; -import org.apache.lucene.codecs.lucene80.Lucene80DocValuesFormat; -import org.apache.lucene.codecs.lucene80.Lucene80NormsFormat; import org.apache.lucene.codecs.lucene86.Lucene86PointsFormat; import org.apache.lucene.codecs.lucene86.Lucene86SegmentInfoFormat; import org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat; @@ -52,16 +50,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, Lucene80DocValuesFormat.Mode.BEST_SPEED), + BEST_SPEED(Lucene87StoredFieldsFormat.Mode.BEST_SPEED, Lucene90DocValuesFormat.Mode.BEST_SPEED), /** Trade retrieval speed for compression ratio. */ BEST_COMPRESSION( Lucene87StoredFieldsFormat.Mode.BEST_COMPRESSION, - Lucene80DocValuesFormat.Mode.BEST_COMPRESSION); + Lucene90DocValuesFormat.Mode.BEST_COMPRESSION); private final Lucene87StoredFieldsFormat.Mode storedMode; - private final Lucene80DocValuesFormat.Mode dvMode; + private final Lucene90DocValuesFormat.Mode dvMode; - private Mode(Lucene87StoredFieldsFormat.Mode storedMode, Lucene80DocValuesFormat.Mode dvMode) { + private Mode(Lucene87StoredFieldsFormat.Mode storedMode, Lucene90DocValuesFormat.Mode dvMode) { this.storedMode = Objects.requireNonNull(storedMode); this.dvMode = Objects.requireNonNull(dvMode); } @@ -109,7 +107,7 @@ public class Lucene90Codec extends Codec { this.storedFieldsFormat = new Lucene87StoredFieldsFormat(Objects.requireNonNull(mode).storedMode); this.defaultFormat = new Lucene90PostingsFormat(); - this.defaultDVFormat = new Lucene80DocValuesFormat(mode.dvMode); + this.defaultDVFormat = new Lucene90DocValuesFormat(mode.dvMode); } @Override @@ -189,7 +187,7 @@ public class Lucene90Codec extends Codec { private final DocValuesFormat defaultDVFormat; - private final NormsFormat normsFormat = new Lucene80NormsFormat(); + private final NormsFormat normsFormat = new Lucene90NormsFormat(); @Override public final NormsFormat normsFormat() { diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesConsumer.java new file mode 100644 index 00000000000..80a5868e9ba --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesConsumer.java @@ -0,0 +1,1036 @@ +/* + * 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 static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT; +import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.NUMERIC_BLOCK_SHIFT; +import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.NUMERIC_BLOCK_SIZE; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.DocValuesConsumer; +import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.index.BinaryDocValues; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.index.DocValues; +import org.apache.lucene.index.EmptyDocValuesProducer; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.index.SortedDocValues; +import org.apache.lucene.index.SortedNumericDocValues; +import org.apache.lucene.index.SortedSetDocValues; +import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.SortedSetSelector; +import org.apache.lucene.store.ByteArrayDataOutput; +import org.apache.lucene.store.ByteBuffersDataOutput; +import org.apache.lucene.store.ByteBuffersIndexOutput; +import org.apache.lucene.store.ChecksumIndexInput; +import org.apache.lucene.store.DataOutput; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefBuilder; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.MathUtil; +import org.apache.lucene.util.StringHelper; +import org.apache.lucene.util.compress.LZ4; +import org.apache.lucene.util.compress.LZ4.FastCompressionHashTable; +import org.apache.lucene.util.packed.DirectMonotonicWriter; +import org.apache.lucene.util.packed.DirectWriter; + +/** writer for {@link Lucene90DocValuesFormat} */ +final class Lucene90DocValuesConsumer extends DocValuesConsumer implements Closeable { + + final Lucene90DocValuesFormat.Mode mode; + IndexOutput data, meta; + final int maxDoc; + private final SegmentWriteState state; + private byte[] termsDictBuffer; + + /** expert: Creates a new writer */ + public Lucene90DocValuesConsumer( + SegmentWriteState state, + String dataCodec, + String dataExtension, + String metaCodec, + String metaExtension, + Lucene90DocValuesFormat.Mode mode) + throws IOException { + this.mode = mode; + if (Lucene90DocValuesFormat.Mode.BEST_COMPRESSION == this.mode) { + this.termsDictBuffer = new byte[1 << 14]; + } + boolean success = false; + try { + this.state = state; + String dataName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, state.segmentSuffix, dataExtension); + data = state.directory.createOutput(dataName, state.context); + CodecUtil.writeIndexHeader( + data, + dataCodec, + Lucene90DocValuesFormat.VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + String metaName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, state.segmentSuffix, metaExtension); + meta = state.directory.createOutput(metaName, state.context); + CodecUtil.writeIndexHeader( + meta, + metaCodec, + Lucene90DocValuesFormat.VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + maxDoc = state.segmentInfo.maxDoc(); + success = true; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(this); + } + } + } + + @Override + public void close() throws IOException { + boolean success = false; + try { + if (meta != null) { + meta.writeInt(-1); // write EOF marker + CodecUtil.writeFooter(meta); // write checksum + } + if (data != null) { + CodecUtil.writeFooter(data); // write checksum + } + success = true; + } finally { + if (success) { + IOUtils.close(data, meta); + } else { + IOUtils.closeWhileHandlingException(data, meta); + } + meta = data = null; + } + } + + @Override + public void addNumericField(FieldInfo field, DocValuesProducer valuesProducer) + throws IOException { + meta.writeInt(field.number); + meta.writeByte(Lucene90DocValuesFormat.NUMERIC); + + writeValues( + field, + new EmptyDocValuesProducer() { + @Override + public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException { + return DocValues.singleton(valuesProducer.getNumeric(field)); + } + }); + } + + private static class MinMaxTracker { + long min, max, numValues, spaceInBits; + + MinMaxTracker() { + reset(); + spaceInBits = 0; + } + + private void reset() { + min = Long.MAX_VALUE; + max = Long.MIN_VALUE; + numValues = 0; + } + + /** Accumulate a new value. */ + void update(long v) { + min = Math.min(min, v); + max = Math.max(max, v); + ++numValues; + } + + /** Update the required space. */ + void finish() { + if (max > min) { + spaceInBits += DirectWriter.unsignedBitsRequired(max - min) * numValues; + } + } + + /** Update space usage and get ready for accumulating values for the next block. */ + void nextBlock() { + finish(); + reset(); + } + } + + private long[] writeValues(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { + SortedNumericDocValues values = valuesProducer.getSortedNumeric(field); + int numDocsWithValue = 0; + MinMaxTracker minMax = new MinMaxTracker(); + MinMaxTracker blockMinMax = new MinMaxTracker(); + long gcd = 0; + Set uniqueValues = new HashSet<>(); + for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) { + for (int i = 0, count = values.docValueCount(); i < count; ++i) { + long v = values.nextValue(); + + if (gcd != 1) { + if (v < Long.MIN_VALUE / 2 || v > Long.MAX_VALUE / 2) { + // in that case v - minValue might overflow and make the GCD computation return + // wrong results. Since these extreme values are unlikely, we just discard + // GCD computation for them + gcd = 1; + } else if (minMax.numValues != 0) { // minValue needs to be set first + gcd = MathUtil.gcd(gcd, v - minMax.min); + } + } + + minMax.update(v); + blockMinMax.update(v); + if (blockMinMax.numValues == NUMERIC_BLOCK_SIZE) { + blockMinMax.nextBlock(); + } + + if (uniqueValues != null && uniqueValues.add(v) && uniqueValues.size() > 256) { + uniqueValues = null; + } + } + + numDocsWithValue++; + } + + minMax.finish(); + blockMinMax.finish(); + + final long numValues = minMax.numValues; + long min = minMax.min; + final long max = minMax.max; + assert blockMinMax.spaceInBits <= minMax.spaceInBits; + + if (numDocsWithValue == 0) { // meta[-2, 0]: No documents with values + meta.writeLong(-2); // docsWithFieldOffset + meta.writeLong(0L); // docsWithFieldLength + meta.writeShort((short) -1); // jumpTableEntryCount + meta.writeByte((byte) -1); // denseRankPower + } else if (numDocsWithValue == maxDoc) { // meta[-1, 0]: All documents has values + meta.writeLong(-1); // docsWithFieldOffset + meta.writeLong(0L); // docsWithFieldLength + meta.writeShort((short) -1); // jumpTableEntryCount + meta.writeByte((byte) -1); // denseRankPower + } else { // meta[data.offset, data.length]: IndexedDISI structure for documents with values + long offset = data.getFilePointer(); + meta.writeLong(offset); // docsWithFieldOffset + values = valuesProducer.getSortedNumeric(field); + final short jumpTableEntryCount = + IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER); + meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength + meta.writeShort(jumpTableEntryCount); + meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER); + } + + meta.writeLong(numValues); + final int numBitsPerValue; + boolean doBlocks = false; + Map encode = null; + if (min >= max) { // meta[-1]: All values are 0 + numBitsPerValue = 0; + meta.writeInt(-1); // tablesize + } else { + if (uniqueValues != null + && uniqueValues.size() > 1 + && DirectWriter.unsignedBitsRequired(uniqueValues.size() - 1) + < DirectWriter.unsignedBitsRequired((max - min) / gcd)) { + numBitsPerValue = DirectWriter.unsignedBitsRequired(uniqueValues.size() - 1); + final Long[] sortedUniqueValues = uniqueValues.toArray(new Long[0]); + Arrays.sort(sortedUniqueValues); + meta.writeInt(sortedUniqueValues.length); // tablesize + for (Long v : sortedUniqueValues) { + meta.writeLong(v); // table[] entry + } + encode = new HashMap<>(); + for (int i = 0; i < sortedUniqueValues.length; ++i) { + encode.put(sortedUniqueValues[i], i); + } + min = 0; + gcd = 1; + } else { + uniqueValues = null; + // we do blocks if that appears to save 10+% storage + doBlocks = + minMax.spaceInBits > 0 && (double) blockMinMax.spaceInBits / minMax.spaceInBits <= 0.9; + if (doBlocks) { + numBitsPerValue = 0xFF; + meta.writeInt(-2 - NUMERIC_BLOCK_SHIFT); // tablesize + } else { + numBitsPerValue = DirectWriter.unsignedBitsRequired((max - min) / gcd); + if (gcd == 1 + && min > 0 + && DirectWriter.unsignedBitsRequired(max) + == DirectWriter.unsignedBitsRequired(max - min)) { + min = 0; + } + meta.writeInt(-1); // tablesize + } + } + } + + meta.writeByte((byte) numBitsPerValue); + meta.writeLong(min); + meta.writeLong(gcd); + long startOffset = data.getFilePointer(); + meta.writeLong(startOffset); // valueOffset + long jumpTableOffset = -1; + if (doBlocks) { + jumpTableOffset = writeValuesMultipleBlocks(valuesProducer.getSortedNumeric(field), gcd); + } else if (numBitsPerValue != 0) { + writeValuesSingleBlock( + valuesProducer.getSortedNumeric(field), numValues, numBitsPerValue, min, gcd, encode); + } + meta.writeLong(data.getFilePointer() - startOffset); // valuesLength + meta.writeLong(jumpTableOffset); + return new long[] {numDocsWithValue, numValues}; + } + + private void writeValuesSingleBlock( + SortedNumericDocValues values, + long numValues, + int numBitsPerValue, + long min, + long gcd, + Map encode) + throws IOException { + DirectWriter writer = DirectWriter.getInstance(data, numValues, numBitsPerValue); + for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) { + for (int i = 0, count = values.docValueCount(); i < count; ++i) { + long v = values.nextValue(); + if (encode == null) { + writer.add((v - min) / gcd); + } else { + writer.add(encode.get(v)); + } + } + } + writer.finish(); + } + + // Returns the offset to the jump-table for vBPV + private long writeValuesMultipleBlocks(SortedNumericDocValues values, long gcd) + throws IOException { + long[] offsets = new long[ArrayUtil.oversize(1, Long.BYTES)]; + int offsetsIndex = 0; + final long[] buffer = new long[NUMERIC_BLOCK_SIZE]; + final ByteBuffersDataOutput encodeBuffer = ByteBuffersDataOutput.newResettableInstance(); + int upTo = 0; + for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) { + for (int i = 0, count = values.docValueCount(); i < count; ++i) { + buffer[upTo++] = values.nextValue(); + if (upTo == NUMERIC_BLOCK_SIZE) { + offsets = ArrayUtil.grow(offsets, offsetsIndex + 1); + offsets[offsetsIndex++] = data.getFilePointer(); + writeBlock(buffer, NUMERIC_BLOCK_SIZE, gcd, encodeBuffer); + upTo = 0; + } + } + } + if (upTo > 0) { + offsets = ArrayUtil.grow(offsets, offsetsIndex + 1); + offsets[offsetsIndex++] = data.getFilePointer(); + writeBlock(buffer, upTo, gcd, encodeBuffer); + } + + // All blocks has been written. Flush the offset jump-table + final long offsetsOrigo = data.getFilePointer(); + for (int i = 0; i < offsetsIndex; i++) { + data.writeLong(offsets[i]); + } + data.writeLong(offsetsOrigo); + return offsetsOrigo; + } + + private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutput buffer) + throws IOException { + assert length > 0; + long min = values[0]; + long max = values[0]; + for (int i = 1; i < length; ++i) { + final long v = values[i]; + assert Math.floorMod(values[i] - min, gcd) == 0; + min = Math.min(min, v); + max = Math.max(max, v); + } + if (min == max) { + data.writeByte((byte) 0); + data.writeLong(min); + } else { + final int bitsPerValue = DirectWriter.unsignedBitsRequired(max - min); + buffer.reset(); + assert buffer.size() == 0; + final DirectWriter w = DirectWriter.getInstance(buffer, length, bitsPerValue); + for (int i = 0; i < length; ++i) { + w.add((values[i] - min) / gcd); + } + w.finish(); + data.writeByte((byte) bitsPerValue); + data.writeLong(min); + data.writeInt(Math.toIntExact(buffer.size())); + buffer.copyTo(data); + } + } + + class CompressedBinaryBlockWriter implements Closeable { + final FastCompressionHashTable ht = new LZ4.FastCompressionHashTable(); + int uncompressedBlockLength = 0; + int maxUncompressedBlockLength = 0; + int numDocsInCurrentBlock = 0; + final int[] docLengths = new int[Lucene90DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK]; + byte[] block = BytesRef.EMPTY_BYTES; + int totalChunks = 0; + long maxPointer = 0; + final long blockAddressesStart; + + private final IndexOutput tempBinaryOffsets; + + public CompressedBinaryBlockWriter() throws IOException { + tempBinaryOffsets = + state.directory.createTempOutput( + state.segmentInfo.name, "binary_pointers", state.context); + boolean success = false; + try { + CodecUtil.writeHeader( + tempBinaryOffsets, + Lucene90DocValuesFormat.META_CODEC + "FilePointers", + Lucene90DocValuesFormat.VERSION_CURRENT); + blockAddressesStart = data.getFilePointer(); + success = true; + } finally { + if (success == false) { + IOUtils.closeWhileHandlingException(this); // self-close because constructor caller can't + } + } + } + + void addDoc(int doc, BytesRef v) throws IOException { + docLengths[numDocsInCurrentBlock] = v.length; + block = ArrayUtil.grow(block, uncompressedBlockLength + v.length); + System.arraycopy(v.bytes, v.offset, block, uncompressedBlockLength, v.length); + uncompressedBlockLength += v.length; + numDocsInCurrentBlock++; + if (numDocsInCurrentBlock == Lucene90DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK) { + flushData(); + } + } + + private void flushData() throws IOException { + if (numDocsInCurrentBlock > 0) { + // Write offset to this block to temporary offsets file + totalChunks++; + long thisBlockStartPointer = data.getFilePointer(); + + // Optimisation - check if all lengths are same + boolean allLengthsSame = true; + for (int i = 1; i < Lucene90DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK; i++) { + if (docLengths[i] != docLengths[i - 1]) { + allLengthsSame = false; + break; + } + } + if (allLengthsSame) { + // Only write one value shifted. Steal a bit to indicate all other lengths are the same + int onlyOneLength = (docLengths[0] << 1) | 1; + data.writeVInt(onlyOneLength); + } else { + for (int i = 0; i < Lucene90DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK; i++) { + if (i == 0) { + // Write first value shifted and steal a bit to indicate other lengths are to follow + int multipleLengths = (docLengths[0] << 1); + data.writeVInt(multipleLengths); + } else { + data.writeVInt(docLengths[i]); + } + } + } + maxUncompressedBlockLength = Math.max(maxUncompressedBlockLength, uncompressedBlockLength); + LZ4.compress(block, 0, uncompressedBlockLength, data, ht); + numDocsInCurrentBlock = 0; + // Ensure initialized with zeroes because full array is always written + Arrays.fill(docLengths, 0); + uncompressedBlockLength = 0; + maxPointer = data.getFilePointer(); + tempBinaryOffsets.writeVLong(maxPointer - thisBlockStartPointer); + } + } + + void writeMetaData() throws IOException { + if (totalChunks == 0) { + return; + } + + long startDMW = data.getFilePointer(); + meta.writeLong(startDMW); + + meta.writeVInt(totalChunks); + meta.writeVInt(Lucene90DocValuesFormat.BINARY_BLOCK_SHIFT); + meta.writeVInt(maxUncompressedBlockLength); + meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT); + + CodecUtil.writeFooter(tempBinaryOffsets); + IOUtils.close(tempBinaryOffsets); + // write the compressed block offsets info to the meta file by reading from temp file + try (ChecksumIndexInput filePointersIn = + state.directory.openChecksumInput(tempBinaryOffsets.getName(), IOContext.READONCE)) { + CodecUtil.checkHeader( + filePointersIn, + Lucene90DocValuesFormat.META_CODEC + "FilePointers", + Lucene90DocValuesFormat.VERSION_CURRENT, + Lucene90DocValuesFormat.VERSION_CURRENT); + Throwable priorE = null; + try { + final DirectMonotonicWriter filePointers = + DirectMonotonicWriter.getInstance( + meta, data, totalChunks, DIRECT_MONOTONIC_BLOCK_SHIFT); + long fp = blockAddressesStart; + for (int i = 0; i < totalChunks; ++i) { + filePointers.add(fp); + fp += filePointersIn.readVLong(); + } + if (maxPointer < fp) { + throw new CorruptIndexException( + "File pointers don't add up (" + fp + " vs expected " + maxPointer + ")", + filePointersIn); + } + filePointers.finish(); + } catch (Throwable e) { + priorE = e; + } finally { + CodecUtil.checkFooter(filePointersIn, priorE); + } + } + // Write the length of the DMW block in the data + meta.writeLong(data.getFilePointer() - startDMW); + } + + @Override + public void close() throws IOException { + if (tempBinaryOffsets != null) { + IOUtils.close(tempBinaryOffsets); + state.directory.deleteFile(tempBinaryOffsets.getName()); + } + } + } + + @Override + public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { + field.putAttribute(Lucene90DocValuesFormat.MODE_KEY, mode.name()); + meta.writeInt(field.number); + meta.writeByte(Lucene90DocValuesFormat.BINARY); + + switch (mode) { + case BEST_SPEED: + doAddUncompressedBinaryField(field, valuesProducer); + break; + case BEST_COMPRESSION: + doAddCompressedBinaryField(field, valuesProducer); + break; + default: + throw new AssertionError(); + } + } + + private void doAddUncompressedBinaryField(FieldInfo field, DocValuesProducer valuesProducer) + throws IOException { + BinaryDocValues values = valuesProducer.getBinary(field); + long start = data.getFilePointer(); + meta.writeLong(start); // dataOffset + int numDocsWithField = 0; + int minLength = Integer.MAX_VALUE; + int maxLength = 0; + for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) { + numDocsWithField++; + BytesRef v = values.binaryValue(); + int length = v.length; + data.writeBytes(v.bytes, v.offset, v.length); + minLength = Math.min(length, minLength); + maxLength = Math.max(length, maxLength); + } + assert numDocsWithField <= maxDoc; + meta.writeLong(data.getFilePointer() - start); // dataLength + + if (numDocsWithField == 0) { + meta.writeLong(-2); // docsWithFieldOffset + meta.writeLong(0L); // docsWithFieldLength + meta.writeShort((short) -1); // jumpTableEntryCount + meta.writeByte((byte) -1); // denseRankPower + } else if (numDocsWithField == maxDoc) { + meta.writeLong(-1); // docsWithFieldOffset + meta.writeLong(0L); // docsWithFieldLength + meta.writeShort((short) -1); // jumpTableEntryCount + meta.writeByte((byte) -1); // denseRankPower + } else { + long offset = data.getFilePointer(); + meta.writeLong(offset); // docsWithFieldOffset + values = valuesProducer.getBinary(field); + final short jumpTableEntryCount = + IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER); + meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength + meta.writeShort(jumpTableEntryCount); + meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER); + } + + meta.writeInt(numDocsWithField); + meta.writeInt(minLength); + meta.writeInt(maxLength); + if (maxLength > minLength) { + start = data.getFilePointer(); + meta.writeLong(start); + meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT); + + final DirectMonotonicWriter writer = + DirectMonotonicWriter.getInstance( + meta, data, numDocsWithField + 1, DIRECT_MONOTONIC_BLOCK_SHIFT); + long addr = 0; + writer.add(addr); + values = valuesProducer.getBinary(field); + for (int doc = values.nextDoc(); + doc != DocIdSetIterator.NO_MORE_DOCS; + doc = values.nextDoc()) { + addr += values.binaryValue().length; + writer.add(addr); + } + writer.finish(); + meta.writeLong(data.getFilePointer() - start); + } + } + + private void doAddCompressedBinaryField(FieldInfo field, DocValuesProducer valuesProducer) + throws IOException { + try (CompressedBinaryBlockWriter blockWriter = new CompressedBinaryBlockWriter()) { + BinaryDocValues values = valuesProducer.getBinary(field); + long start = data.getFilePointer(); + meta.writeLong(start); // dataOffset + int numDocsWithField = 0; + int minLength = Integer.MAX_VALUE; + int maxLength = 0; + for (int doc = values.nextDoc(); + doc != DocIdSetIterator.NO_MORE_DOCS; + doc = values.nextDoc()) { + numDocsWithField++; + BytesRef v = values.binaryValue(); + blockWriter.addDoc(doc, v); + int length = v.length; + minLength = Math.min(length, minLength); + maxLength = Math.max(length, maxLength); + } + blockWriter.flushData(); + + assert numDocsWithField <= maxDoc; + meta.writeLong(data.getFilePointer() - start); // dataLength + + if (numDocsWithField == 0) { + meta.writeLong(-2); // docsWithFieldOffset + meta.writeLong(0L); // docsWithFieldLength + meta.writeShort((short) -1); // jumpTableEntryCount + meta.writeByte((byte) -1); // denseRankPower + } else if (numDocsWithField == maxDoc) { + meta.writeLong(-1); // docsWithFieldOffset + meta.writeLong(0L); // docsWithFieldLength + meta.writeShort((short) -1); // jumpTableEntryCount + meta.writeByte((byte) -1); // denseRankPower + } else { + long offset = data.getFilePointer(); + meta.writeLong(offset); // docsWithFieldOffset + values = valuesProducer.getBinary(field); + final short jumpTableEntryCount = + IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER); + meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength + meta.writeShort(jumpTableEntryCount); + meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER); + } + + meta.writeInt(numDocsWithField); + meta.writeInt(minLength); + meta.writeInt(maxLength); + + blockWriter.writeMetaData(); + } + } + + @Override + public void addSortedField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { + meta.writeInt(field.number); + meta.writeByte(Lucene90DocValuesFormat.SORTED); + doAddSortedField(field, valuesProducer); + } + + private void doAddSortedField(FieldInfo field, DocValuesProducer valuesProducer) + throws IOException { + SortedDocValues values = valuesProducer.getSorted(field); + int numDocsWithField = 0; + for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) { + numDocsWithField++; + } + + if (numDocsWithField == 0) { + meta.writeLong(-2); // docsWithFieldOffset + meta.writeLong(0L); // docsWithFieldLength + meta.writeShort((short) -1); // jumpTableEntryCount + meta.writeByte((byte) -1); // denseRankPower + } else if (numDocsWithField == maxDoc) { + meta.writeLong(-1); // docsWithFieldOffset + meta.writeLong(0L); // docsWithFieldLength + meta.writeShort((short) -1); // jumpTableEntryCount + meta.writeByte((byte) -1); // denseRankPower + } else { + long offset = data.getFilePointer(); + meta.writeLong(offset); // docsWithFieldOffset + values = valuesProducer.getSorted(field); + final short jumpTableentryCount = + IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER); + meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength + meta.writeShort(jumpTableentryCount); + meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER); + } + + meta.writeInt(numDocsWithField); + if (values.getValueCount() <= 1) { + meta.writeByte((byte) 0); // bitsPerValue + meta.writeLong(0L); // ordsOffset + meta.writeLong(0L); // ordsLength + } else { + int numberOfBitsPerOrd = DirectWriter.unsignedBitsRequired(values.getValueCount() - 1); + meta.writeByte((byte) numberOfBitsPerOrd); // bitsPerValue + long start = data.getFilePointer(); + meta.writeLong(start); // ordsOffset + DirectWriter writer = DirectWriter.getInstance(data, numDocsWithField, numberOfBitsPerOrd); + values = valuesProducer.getSorted(field); + for (int doc = values.nextDoc(); + doc != DocIdSetIterator.NO_MORE_DOCS; + doc = values.nextDoc()) { + writer.add(values.ordValue()); + } + writer.finish(); + meta.writeLong(data.getFilePointer() - start); // ordsLength + } + + addTermsDict(DocValues.singleton(valuesProducer.getSorted(field))); + } + + private void addTermsDict(SortedSetDocValues values) throws IOException { + final long size = values.getValueCount(); + meta.writeVLong(size); + boolean compress = + Lucene90DocValuesFormat.Mode.BEST_COMPRESSION == mode + && values.getValueCount() + > Lucene90DocValuesFormat.TERMS_DICT_BLOCK_COMPRESSION_THRESHOLD; + int code, blockMask, shift; + if (compress) { + code = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_CODE; + blockMask = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_MASK; + shift = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_SHIFT; + } else { + code = shift = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_SHIFT; + blockMask = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_MASK; + } + + meta.writeInt(code); + meta.writeInt(DIRECT_MONOTONIC_BLOCK_SHIFT); + ByteBuffersDataOutput addressBuffer = new ByteBuffersDataOutput(); + ByteBuffersIndexOutput addressOutput = + new ByteBuffersIndexOutput(addressBuffer, "temp", "temp"); + long numBlocks = (size + blockMask) >>> shift; + DirectMonotonicWriter writer = + DirectMonotonicWriter.getInstance( + meta, addressOutput, numBlocks, DIRECT_MONOTONIC_BLOCK_SHIFT); + + BytesRefBuilder previous = new BytesRefBuilder(); + long ord = 0; + long start = data.getFilePointer(); + int maxLength = 0, maxBlockLength = 0; + TermsEnum iterator = values.termsEnum(); + + LZ4.FastCompressionHashTable ht = null; + ByteArrayDataOutput bufferedOutput = null; + if (compress) { + ht = new LZ4.FastCompressionHashTable(); + bufferedOutput = new ByteArrayDataOutput(termsDictBuffer); + } + + for (BytesRef term = iterator.next(); term != null; term = iterator.next()) { + if ((ord & blockMask) == 0) { + if (compress && bufferedOutput.getPosition() > 0) { + maxBlockLength = + Math.max(maxBlockLength, compressAndGetTermsDictBlockLength(bufferedOutput, ht)); + bufferedOutput.reset(termsDictBuffer); + } + + writer.add(data.getFilePointer() - start); + data.writeVInt(term.length); + data.writeBytes(term.bytes, term.offset, term.length); + } else { + final int prefixLength = StringHelper.bytesDifference(previous.get(), term); + final int suffixLength = term.length - prefixLength; + assert suffixLength > 0; // terms are unique + DataOutput blockOutput; + if (compress) { + // Will write (suffixLength + 1 byte + 2 vint) bytes. Grow the buffer in need. + bufferedOutput = maybeGrowBuffer(bufferedOutput, suffixLength + 11); + blockOutput = bufferedOutput; + } else { + blockOutput = data; + } + blockOutput.writeByte( + (byte) (Math.min(prefixLength, 15) | (Math.min(15, suffixLength - 1) << 4))); + if (prefixLength >= 15) { + blockOutput.writeVInt(prefixLength - 15); + } + if (suffixLength >= 16) { + blockOutput.writeVInt(suffixLength - 16); + } + blockOutput.writeBytes(term.bytes, term.offset + prefixLength, suffixLength); + } + maxLength = Math.max(maxLength, term.length); + previous.copyBytes(term); + ++ord; + } + // Compress and write out the last block + if (compress && bufferedOutput.getPosition() > 0) { + maxBlockLength = + Math.max(maxBlockLength, compressAndGetTermsDictBlockLength(bufferedOutput, ht)); + } + + writer.finish(); + meta.writeInt(maxLength); + if (compress) { + // Write one more int for storing max block length. For compressed terms dict only. + meta.writeInt(maxBlockLength); + } + meta.writeLong(start); + meta.writeLong(data.getFilePointer() - start); + start = data.getFilePointer(); + addressBuffer.copyTo(data); + meta.writeLong(start); + meta.writeLong(data.getFilePointer() - start); + + // Now write the reverse terms index + writeTermsIndex(values); + } + + private int compressAndGetTermsDictBlockLength( + ByteArrayDataOutput bufferedOutput, LZ4.FastCompressionHashTable ht) throws IOException { + int uncompressedLength = bufferedOutput.getPosition(); + data.writeVInt(uncompressedLength); + long before = data.getFilePointer(); + LZ4.compress(termsDictBuffer, 0, uncompressedLength, data, ht); + int compressedLength = (int) (data.getFilePointer() - before); + // Block length will be used for creating buffer for decompression, one corner case is that + // compressed length might be bigger than un-compressed length, so just return the bigger one. + return Math.max(uncompressedLength, compressedLength); + } + + private ByteArrayDataOutput maybeGrowBuffer(ByteArrayDataOutput bufferedOutput, int termLength) { + int pos = bufferedOutput.getPosition(), originalLength = termsDictBuffer.length; + if (pos + termLength >= originalLength - 1) { + termsDictBuffer = ArrayUtil.grow(termsDictBuffer, originalLength + termLength); + bufferedOutput = new ByteArrayDataOutput(termsDictBuffer, pos, termsDictBuffer.length - pos); + } + return bufferedOutput; + } + + private void writeTermsIndex(SortedSetDocValues values) throws IOException { + final long size = values.getValueCount(); + meta.writeInt(Lucene90DocValuesFormat.TERMS_DICT_REVERSE_INDEX_SHIFT); + long start = data.getFilePointer(); + + long numBlocks = + 1L + + ((size + Lucene90DocValuesFormat.TERMS_DICT_REVERSE_INDEX_MASK) + >>> Lucene90DocValuesFormat.TERMS_DICT_REVERSE_INDEX_SHIFT); + ByteBuffersDataOutput addressBuffer = new ByteBuffersDataOutput(); + DirectMonotonicWriter writer; + try (ByteBuffersIndexOutput addressOutput = + new ByteBuffersIndexOutput(addressBuffer, "temp", "temp")) { + writer = + DirectMonotonicWriter.getInstance( + meta, addressOutput, numBlocks, DIRECT_MONOTONIC_BLOCK_SHIFT); + TermsEnum iterator = values.termsEnum(); + BytesRefBuilder previous = new BytesRefBuilder(); + long offset = 0; + long ord = 0; + for (BytesRef term = iterator.next(); term != null; term = iterator.next()) { + if ((ord & Lucene90DocValuesFormat.TERMS_DICT_REVERSE_INDEX_MASK) == 0) { + writer.add(offset); + final int sortKeyLength; + if (ord == 0) { + // no previous term: no bytes to write + sortKeyLength = 0; + } else { + sortKeyLength = StringHelper.sortKeyLength(previous.get(), term); + } + offset += sortKeyLength; + data.writeBytes(term.bytes, term.offset, sortKeyLength); + } else if ((ord & Lucene90DocValuesFormat.TERMS_DICT_REVERSE_INDEX_MASK) + == Lucene90DocValuesFormat.TERMS_DICT_REVERSE_INDEX_MASK) { + previous.copyBytes(term); + } + ++ord; + } + writer.add(offset); + writer.finish(); + meta.writeLong(start); + meta.writeLong(data.getFilePointer() - start); + start = data.getFilePointer(); + addressBuffer.copyTo(data); + meta.writeLong(start); + meta.writeLong(data.getFilePointer() - start); + } + } + + @Override + public void addSortedNumericField(FieldInfo field, DocValuesProducer valuesProducer) + throws IOException { + meta.writeInt(field.number); + meta.writeByte(Lucene90DocValuesFormat.SORTED_NUMERIC); + + long[] stats = writeValues(field, valuesProducer); + int numDocsWithField = Math.toIntExact(stats[0]); + long numValues = stats[1]; + assert numValues >= numDocsWithField; + + meta.writeInt(numDocsWithField); + if (numValues > numDocsWithField) { + long start = data.getFilePointer(); + meta.writeLong(start); + meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT); + + final DirectMonotonicWriter addressesWriter = + DirectMonotonicWriter.getInstance( + meta, data, numDocsWithField + 1L, DIRECT_MONOTONIC_BLOCK_SHIFT); + long addr = 0; + addressesWriter.add(addr); + SortedNumericDocValues values = valuesProducer.getSortedNumeric(field); + for (int doc = values.nextDoc(); + doc != DocIdSetIterator.NO_MORE_DOCS; + doc = values.nextDoc()) { + addr += values.docValueCount(); + addressesWriter.add(addr); + } + addressesWriter.finish(); + meta.writeLong(data.getFilePointer() - start); + } + } + + @Override + public void addSortedSetField(FieldInfo field, DocValuesProducer valuesProducer) + throws IOException { + meta.writeInt(field.number); + meta.writeByte(Lucene90DocValuesFormat.SORTED_SET); + + SortedSetDocValues values = valuesProducer.getSortedSet(field); + int numDocsWithField = 0; + long numOrds = 0; + for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) { + numDocsWithField++; + for (long ord = values.nextOrd(); + ord != SortedSetDocValues.NO_MORE_ORDS; + ord = values.nextOrd()) { + numOrds++; + } + } + + if (numDocsWithField == numOrds) { + meta.writeByte((byte) 0); // multiValued (0 = singleValued) + doAddSortedField( + field, + new EmptyDocValuesProducer() { + @Override + public SortedDocValues getSorted(FieldInfo field) throws IOException { + return SortedSetSelector.wrap( + valuesProducer.getSortedSet(field), SortedSetSelector.Type.MIN); + } + }); + return; + } + meta.writeByte((byte) 1); // multiValued (1 = multiValued) + + assert numDocsWithField != 0; + if (numDocsWithField == maxDoc) { + meta.writeLong(-1); // docsWithFieldOffset + meta.writeLong(0L); // docsWithFieldLength + meta.writeShort((short) -1); // jumpTableEntryCount + meta.writeByte((byte) -1); // denseRankPower + } else { + long offset = data.getFilePointer(); + meta.writeLong(offset); // docsWithFieldOffset + values = valuesProducer.getSortedSet(field); + final short jumpTableEntryCount = + IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER); + meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength + meta.writeShort(jumpTableEntryCount); + meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER); + } + + int numberOfBitsPerOrd = DirectWriter.unsignedBitsRequired(values.getValueCount() - 1); + meta.writeByte((byte) numberOfBitsPerOrd); // bitsPerValue + long start = data.getFilePointer(); + meta.writeLong(start); // ordsOffset + DirectWriter writer = DirectWriter.getInstance(data, numOrds, numberOfBitsPerOrd); + values = valuesProducer.getSortedSet(field); + for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) { + for (long ord = values.nextOrd(); + ord != SortedSetDocValues.NO_MORE_ORDS; + ord = values.nextOrd()) { + writer.add(ord); + } + } + writer.finish(); + meta.writeLong(data.getFilePointer() - start); // ordsLength + + meta.writeInt(numDocsWithField); + start = data.getFilePointer(); + meta.writeLong(start); // addressesOffset + meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT); + + final DirectMonotonicWriter addressesWriter = + DirectMonotonicWriter.getInstance( + meta, data, numDocsWithField + 1, DIRECT_MONOTONIC_BLOCK_SHIFT); + long addr = 0; + addressesWriter.add(addr); + values = valuesProducer.getSortedSet(field); + for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) { + values.nextOrd(); + addr++; + while (values.nextOrd() != SortedSetDocValues.NO_MORE_ORDS) { + addr++; + } + addressesWriter.add(addr); + } + addressesWriter.finish(); + meta.writeLong(data.getFilePointer() - start); // addressesLength + + addTermsDict(values); + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesFormat.java new file mode 100644 index 00000000000..51db858deb7 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesFormat.java @@ -0,0 +1,216 @@ +/* + * 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.DocValuesConsumer; +import org.apache.lucene.codecs.DocValuesFormat; +import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.index.DocValuesType; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.store.DataOutput; +import org.apache.lucene.util.SmallFloat; +import org.apache.lucene.util.packed.DirectWriter; + +/** + * Lucene 9.0 DocValues format. + * + *

Documents that have a value for the field are encoded in a way that it is always possible to + * know the ordinal of the current document in the set of documents that have a value. For instance, + * say the set of documents that have a value for the field is {1, 5, 6, 11}. When the + * iterator is on 6, it knows that this is the 3rd item of the set. This way, values + * can be stored densely and accessed based on their index at search time. If all documents in a + * segment have a value for the field, the index is the same as the doc ID, so this case is encoded + * implicitly and is very fast at query time. On the other hand if some documents are missing a + * value for the field then the set of documents that have a value is encoded into blocks. All doc + * IDs that share the same upper 16 bits are encoded into the same block with the following + * strategies: + * + *

+ * + *

Skipping blocks to arrive at a wanted document is either done on an iterative basis or by + * using the jump-table stored at the end of the chain of blocks. The jump-table holds the offset as + * well as the index for all blocks, packed in a single long per block. + * + *

Then the five per-document value types (Numeric,Binary,Sorted,SortedSet,SortedNumeric) are + * encoded using the following strategies: + * + *

{@link DocValuesType#NUMERIC NUMERIC}: + * + *

+ * + *

Depending on calculated gains, the numbers might be split into blocks of 16384 values. In that + * case, a jump-table with block offsets is appended to the blocks for O(1) access to the needed + * block. + * + *

{@link DocValuesType#BINARY BINARY}: + * + *

+ * + *

{@link DocValuesType#SORTED SORTED}: + * + *

+ * + *

{@link DocValuesType#SORTED_SET SORTED_SET}: + * + *

+ * + *

{@link DocValuesType#SORTED_NUMERIC SORTED_NUMERIC}: + * + *

+ * + *

Files: + * + *

    + *
  1. .dvd: DocValues data + *
  2. .dvm: DocValues metadata + *
+ * + * @lucene.experimental + */ +public final class Lucene90DocValuesFormat extends DocValuesFormat { + + /** Configuration option for doc values. */ + 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 = Lucene90DocValuesFormat.class.getSimpleName() + ".mode"; + + private final Mode mode; + + /** Default constructor. */ + public Lucene90DocValuesFormat() { + this(Mode.BEST_SPEED); + } + + /** Constructor */ + public Lucene90DocValuesFormat(Mode mode) { + super("Lucene90"); + this.mode = Objects.requireNonNull(mode); + } + + @Override + public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException { + return new Lucene90DocValuesConsumer( + state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION, mode); + } + + @Override + public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException { + return new Lucene90DocValuesProducer( + state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION); + } + + static final String DATA_CODEC = "Lucene90DocValuesData"; + static final String DATA_EXTENSION = "dvd"; + static final String META_CODEC = "Lucene90DocValuesMetadata"; + static final String META_EXTENSION = "dvm"; + static final int VERSION_START = 0; + static final int VERSION_CURRENT = VERSION_START; + + // indicates docvalues type + static final byte NUMERIC = 0; + static final byte BINARY = 1; + static final byte SORTED = 2; + static final byte SORTED_SET = 3; + static final byte SORTED_NUMERIC = 4; + + static final int DIRECT_MONOTONIC_BLOCK_SHIFT = 16; + + static final int NUMERIC_BLOCK_SHIFT = 14; + static final int NUMERIC_BLOCK_SIZE = 1 << NUMERIC_BLOCK_SHIFT; + + static final int BINARY_BLOCK_SHIFT = 5; + static final int BINARY_DOCS_PER_COMPRESSED_BLOCK = 1 << BINARY_BLOCK_SHIFT; + + static final int TERMS_DICT_BLOCK_SHIFT = 4; + static final int TERMS_DICT_BLOCK_SIZE = 1 << TERMS_DICT_BLOCK_SHIFT; + static final int TERMS_DICT_BLOCK_MASK = TERMS_DICT_BLOCK_SIZE - 1; + + static final int TERMS_DICT_BLOCK_COMPRESSION_THRESHOLD = 32; + static final int TERMS_DICT_BLOCK_LZ4_SHIFT = 6; + static final int TERMS_DICT_BLOCK_LZ4_SIZE = 1 << TERMS_DICT_BLOCK_LZ4_SHIFT; + static final int TERMS_DICT_BLOCK_LZ4_MASK = TERMS_DICT_BLOCK_LZ4_SIZE - 1; + static final int TERMS_DICT_COMPRESSOR_LZ4_CODE = 1; + // Writing a special code so we know this is a LZ4-compressed block. + static final int TERMS_DICT_BLOCK_LZ4_CODE = + TERMS_DICT_BLOCK_LZ4_SHIFT << 16 | TERMS_DICT_COMPRESSOR_LZ4_CODE; + + static final int TERMS_DICT_REVERSE_INDEX_SHIFT = 10; + static final int TERMS_DICT_REVERSE_INDEX_SIZE = 1 << TERMS_DICT_REVERSE_INDEX_SHIFT; + static final int TERMS_DICT_REVERSE_INDEX_MASK = TERMS_DICT_REVERSE_INDEX_SIZE - 1; +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesProducer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesProducer.java new file mode 100644 index 00000000000..0a2bb13be69 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesProducer.java @@ -0,0 +1,1734 @@ +/* + * 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.Closeable; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.index.BaseTermsEnum; +import org.apache.lucene.index.BinaryDocValues; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.index.DocValues; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.ImpactsEnum; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.NumericDocValues; +import org.apache.lucene.index.PostingsEnum; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.index.SortedDocValues; +import org.apache.lucene.index.SortedNumericDocValues; +import org.apache.lucene.index.SortedSetDocValues; +import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.index.TermsEnum.SeekStatus; +import org.apache.lucene.store.ByteArrayDataInput; +import org.apache.lucene.store.ChecksumIndexInput; +import org.apache.lucene.store.DataInput; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.RandomAccessInput; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.LongValues; +import org.apache.lucene.util.RamUsageEstimator; +import org.apache.lucene.util.compress.LZ4; +import org.apache.lucene.util.packed.DirectMonotonicReader; +import org.apache.lucene.util.packed.DirectReader; + +/** reader for {@link Lucene90DocValuesFormat} */ +final class Lucene90DocValuesProducer extends DocValuesProducer implements Closeable { + private final Map numerics = new HashMap<>(); + private final Map binaries = new HashMap<>(); + private final Map sorted = new HashMap<>(); + private final Map sortedSets = new HashMap<>(); + private final Map sortedNumerics = new HashMap<>(); + private long ramBytesUsed; + private final IndexInput data; + private final int maxDoc; + private int version = -1; + + /** expert: instantiates a new reader */ + Lucene90DocValuesProducer( + SegmentReadState state, + String dataCodec, + String dataExtension, + String metaCodec, + String metaExtension) + throws IOException { + String metaName = + IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension); + this.maxDoc = state.segmentInfo.maxDoc(); + ramBytesUsed = RamUsageEstimator.shallowSizeOfInstance(getClass()); + + // read in the entries from the metadata file. + try (ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context)) { + Throwable priorE = null; + + try { + version = + CodecUtil.checkIndexHeader( + in, + metaCodec, + Lucene90DocValuesFormat.VERSION_START, + Lucene90DocValuesFormat.VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + + readFields(state.segmentInfo.name, in, state.fieldInfos); + + } catch (Throwable exception) { + priorE = exception; + } finally { + CodecUtil.checkFooter(in, priorE); + } + } + + String dataName = + IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension); + this.data = state.directory.openInput(dataName, state.context); + boolean success = false; + try { + final int version2 = + CodecUtil.checkIndexHeader( + data, + dataCodec, + Lucene90DocValuesFormat.VERSION_START, + Lucene90DocValuesFormat.VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + if (version != version2) { + throw new CorruptIndexException( + "Format versions mismatch: meta=" + version + ", data=" + version2, data); + } + + // 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(data); + + success = true; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(this.data); + } + } + } + + private void readFields(String segmentName, IndexInput meta, FieldInfos infos) + throws IOException { + for (int fieldNumber = meta.readInt(); fieldNumber != -1; fieldNumber = meta.readInt()) { + FieldInfo info = infos.fieldInfo(fieldNumber); + if (info == null) { + throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta); + } + byte type = meta.readByte(); + if (type == Lucene90DocValuesFormat.NUMERIC) { + numerics.put(info.name, readNumeric(meta)); + } else if (type == Lucene90DocValuesFormat.BINARY) { + String value = info.getAttribute(Lucene90DocValuesFormat.MODE_KEY); + if (value == null) { + throw new IllegalStateException( + "missing value for " + + Lucene90DocValuesFormat.MODE_KEY + + " for field: " + + info.name + + " in segment: " + + segmentName); + } + Lucene90DocValuesFormat.Mode mode = Lucene90DocValuesFormat.Mode.valueOf(value); + final boolean compressed = mode == Lucene90DocValuesFormat.Mode.BEST_COMPRESSION; + binaries.put(info.name, readBinary(meta, compressed)); + } else if (type == Lucene90DocValuesFormat.SORTED) { + sorted.put(info.name, readSorted(meta)); + } else if (type == Lucene90DocValuesFormat.SORTED_SET) { + sortedSets.put(info.name, readSortedSet(meta)); + } else if (type == Lucene90DocValuesFormat.SORTED_NUMERIC) { + sortedNumerics.put(info.name, readSortedNumeric(meta)); + } else { + throw new CorruptIndexException("invalid type: " + type, meta); + } + } + } + + private NumericEntry readNumeric(IndexInput meta) throws IOException { + NumericEntry entry = new NumericEntry(); + readNumeric(meta, entry); + return entry; + } + + private void readNumeric(IndexInput meta, NumericEntry entry) throws IOException { + entry.docsWithFieldOffset = meta.readLong(); + entry.docsWithFieldLength = meta.readLong(); + entry.jumpTableEntryCount = meta.readShort(); + entry.denseRankPower = meta.readByte(); + entry.numValues = meta.readLong(); + int tableSize = meta.readInt(); + if (tableSize > 256) { + throw new CorruptIndexException("invalid table size: " + tableSize, meta); + } + if (tableSize >= 0) { + entry.table = new long[tableSize]; + ramBytesUsed += RamUsageEstimator.sizeOf(entry.table); + for (int i = 0; i < tableSize; ++i) { + entry.table[i] = meta.readLong(); + } + } + if (tableSize < -1) { + entry.blockShift = -2 - tableSize; + } else { + entry.blockShift = -1; + } + entry.bitsPerValue = meta.readByte(); + entry.minValue = meta.readLong(); + entry.gcd = meta.readLong(); + entry.valuesOffset = meta.readLong(); + entry.valuesLength = meta.readLong(); + entry.valueJumpTableOffset = meta.readLong(); + } + + private BinaryEntry readBinary(IndexInput meta, boolean compressed) throws IOException { + final BinaryEntry entry = new BinaryEntry(); + entry.compressed = compressed; + entry.dataOffset = meta.readLong(); + entry.dataLength = meta.readLong(); + entry.docsWithFieldOffset = meta.readLong(); + entry.docsWithFieldLength = meta.readLong(); + entry.jumpTableEntryCount = meta.readShort(); + entry.denseRankPower = meta.readByte(); + entry.numDocsWithField = meta.readInt(); + entry.minLength = meta.readInt(); + entry.maxLength = meta.readInt(); + if ((entry.compressed && entry.numDocsWithField > 0) || entry.minLength < entry.maxLength) { + entry.addressesOffset = meta.readLong(); + + // Old count of uncompressed addresses + long numAddresses = entry.numDocsWithField + 1L; + // New count of compressed addresses - the number of compresseed blocks + if (entry.compressed) { + entry.numCompressedChunks = meta.readVInt(); + entry.docsPerChunkShift = meta.readVInt(); + entry.maxUncompressedChunkSize = meta.readVInt(); + numAddresses = entry.numCompressedChunks; + } + + final int blockShift = meta.readVInt(); + entry.addressesMeta = DirectMonotonicReader.loadMeta(meta, numAddresses, blockShift); + ramBytesUsed += entry.addressesMeta.ramBytesUsed(); + entry.addressesLength = meta.readLong(); + } + return entry; + } + + private SortedEntry readSorted(IndexInput meta) throws IOException { + SortedEntry entry = new SortedEntry(); + entry.docsWithFieldOffset = meta.readLong(); + entry.docsWithFieldLength = meta.readLong(); + entry.jumpTableEntryCount = meta.readShort(); + entry.denseRankPower = meta.readByte(); + entry.numDocsWithField = meta.readInt(); + entry.bitsPerValue = meta.readByte(); + entry.ordsOffset = meta.readLong(); + entry.ordsLength = meta.readLong(); + readTermDict(meta, entry); + return entry; + } + + private SortedSetEntry readSortedSet(IndexInput meta) throws IOException { + SortedSetEntry entry = new SortedSetEntry(); + byte multiValued = meta.readByte(); + switch (multiValued) { + case 0: // singlevalued + entry.singleValueEntry = readSorted(meta); + return entry; + case 1: // multivalued + break; + default: + throw new CorruptIndexException("Invalid multiValued flag: " + multiValued, meta); + } + entry.docsWithFieldOffset = meta.readLong(); + entry.docsWithFieldLength = meta.readLong(); + entry.jumpTableEntryCount = meta.readShort(); + entry.denseRankPower = meta.readByte(); + entry.bitsPerValue = meta.readByte(); + entry.ordsOffset = meta.readLong(); + entry.ordsLength = meta.readLong(); + entry.numDocsWithField = meta.readInt(); + entry.addressesOffset = meta.readLong(); + final int blockShift = meta.readVInt(); + entry.addressesMeta = + DirectMonotonicReader.loadMeta(meta, entry.numDocsWithField + 1, blockShift); + ramBytesUsed += entry.addressesMeta.ramBytesUsed(); + entry.addressesLength = meta.readLong(); + readTermDict(meta, entry); + return entry; + } + + private static void readTermDict(IndexInput meta, TermsDictEntry entry) throws IOException { + entry.termsDictSize = meta.readVLong(); + int termsDictBlockCode = meta.readInt(); + if (Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_CODE == termsDictBlockCode) { + // This is a LZ4 compressed block. + entry.compressed = true; + entry.termsDictBlockShift = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_SHIFT; + } else { + entry.termsDictBlockShift = termsDictBlockCode; + } + + final int blockShift = meta.readInt(); + final long addressesSize = + (entry.termsDictSize + (1L << entry.termsDictBlockShift) - 1) >>> entry.termsDictBlockShift; + entry.termsAddressesMeta = DirectMonotonicReader.loadMeta(meta, addressesSize, blockShift); + entry.maxTermLength = meta.readInt(); + // Read one more int for compressed term dict. + if (entry.compressed) { + entry.maxBlockLength = meta.readInt(); + } + entry.termsDataOffset = meta.readLong(); + entry.termsDataLength = meta.readLong(); + entry.termsAddressesOffset = meta.readLong(); + entry.termsAddressesLength = meta.readLong(); + entry.termsDictIndexShift = meta.readInt(); + final long indexSize = + (entry.termsDictSize + (1L << entry.termsDictIndexShift) - 1) >>> entry.termsDictIndexShift; + entry.termsIndexAddressesMeta = DirectMonotonicReader.loadMeta(meta, 1 + indexSize, blockShift); + entry.termsIndexOffset = meta.readLong(); + entry.termsIndexLength = meta.readLong(); + entry.termsIndexAddressesOffset = meta.readLong(); + entry.termsIndexAddressesLength = meta.readLong(); + } + + private SortedNumericEntry readSortedNumeric(IndexInput meta) throws IOException { + SortedNumericEntry entry = new SortedNumericEntry(); + readNumeric(meta, entry); + entry.numDocsWithField = meta.readInt(); + if (entry.numDocsWithField != entry.numValues) { + entry.addressesOffset = meta.readLong(); + final int blockShift = meta.readVInt(); + entry.addressesMeta = + DirectMonotonicReader.loadMeta(meta, entry.numDocsWithField + 1, blockShift); + ramBytesUsed += entry.addressesMeta.ramBytesUsed(); + entry.addressesLength = meta.readLong(); + } + return entry; + } + + @Override + public void close() throws IOException { + data.close(); + } + + private static class NumericEntry { + long[] table; + int blockShift; + byte bitsPerValue; + long docsWithFieldOffset; + long docsWithFieldLength; + short jumpTableEntryCount; + byte denseRankPower; + long numValues; + long minValue; + long gcd; + long valuesOffset; + long valuesLength; + long valueJumpTableOffset; // -1 if no jump-table + } + + private static class BinaryEntry { + boolean compressed; + long dataOffset; + long dataLength; + long docsWithFieldOffset; + long docsWithFieldLength; + short jumpTableEntryCount; + byte denseRankPower; + int numDocsWithField; + int minLength; + int maxLength; + long addressesOffset; + long addressesLength; + DirectMonotonicReader.Meta addressesMeta; + int numCompressedChunks; + int docsPerChunkShift; + int maxUncompressedChunkSize; + } + + private static class TermsDictEntry { + long termsDictSize; + int termsDictBlockShift; + DirectMonotonicReader.Meta termsAddressesMeta; + int maxTermLength; + long termsDataOffset; + long termsDataLength; + long termsAddressesOffset; + long termsAddressesLength; + int termsDictIndexShift; + DirectMonotonicReader.Meta termsIndexAddressesMeta; + long termsIndexOffset; + long termsIndexLength; + long termsIndexAddressesOffset; + long termsIndexAddressesLength; + + boolean compressed; + int maxBlockLength; + } + + private static class SortedEntry extends TermsDictEntry { + long docsWithFieldOffset; + long docsWithFieldLength; + short jumpTableEntryCount; + byte denseRankPower; + int numDocsWithField; + byte bitsPerValue; + long ordsOffset; + long ordsLength; + } + + private static class SortedSetEntry extends TermsDictEntry { + SortedEntry singleValueEntry; + long docsWithFieldOffset; + long docsWithFieldLength; + short jumpTableEntryCount; + byte denseRankPower; + int numDocsWithField; + byte bitsPerValue; + long ordsOffset; + long ordsLength; + DirectMonotonicReader.Meta addressesMeta; + long addressesOffset; + long addressesLength; + } + + private static class SortedNumericEntry extends NumericEntry { + int numDocsWithField; + DirectMonotonicReader.Meta addressesMeta; + long addressesOffset; + long addressesLength; + } + + @Override + public long ramBytesUsed() { + return ramBytesUsed; + } + + @Override + public NumericDocValues getNumeric(FieldInfo field) throws IOException { + NumericEntry entry = numerics.get(field.name); + return getNumeric(entry); + } + + private abstract static class DenseNumericDocValues extends NumericDocValues { + + final int maxDoc; + int doc = -1; + + DenseNumericDocValues(int maxDoc) { + this.maxDoc = maxDoc; + } + + @Override + public int docID() { + return doc; + } + + @Override + public int nextDoc() throws IOException { + return advance(doc + 1); + } + + @Override + public int advance(int target) throws IOException { + if (target >= maxDoc) { + return doc = NO_MORE_DOCS; + } + return doc = target; + } + + @Override + public boolean advanceExact(int target) { + doc = target; + return true; + } + + @Override + public long cost() { + return maxDoc; + } + } + + private abstract static class SparseNumericDocValues extends NumericDocValues { + + final IndexedDISI disi; + + SparseNumericDocValues(IndexedDISI disi) { + this.disi = disi; + } + + @Override + public int advance(int target) throws IOException { + return disi.advance(target); + } + + @Override + public boolean advanceExact(int target) throws IOException { + return disi.advanceExact(target); + } + + @Override + public int nextDoc() throws IOException { + return disi.nextDoc(); + } + + @Override + public int docID() { + return disi.docID(); + } + + @Override + public long cost() { + return disi.cost(); + } + } + + private NumericDocValues getNumeric(NumericEntry entry) throws IOException { + if (entry.docsWithFieldOffset == -2) { + // empty + return DocValues.emptyNumeric(); + } else if (entry.docsWithFieldOffset == -1) { + // dense + if (entry.bitsPerValue == 0) { + return new DenseNumericDocValues(maxDoc) { + @Override + public long longValue() throws IOException { + return entry.minValue; + } + }; + } else { + final RandomAccessInput slice = + data.randomAccessSlice(entry.valuesOffset, entry.valuesLength); + if (entry.blockShift >= 0) { + // dense but split into blocks of different bits per value + return new DenseNumericDocValues(maxDoc) { + final VaryingBPVReader vBPVReader = new VaryingBPVReader(entry, slice); + + @Override + public long longValue() throws IOException { + return vBPVReader.getLongValue(doc); + } + }; + } else { + final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue); + if (entry.table != null) { + final long[] table = entry.table; + return new DenseNumericDocValues(maxDoc) { + @Override + public long longValue() throws IOException { + return table[(int) values.get(doc)]; + } + }; + } else { + final long mul = entry.gcd; + final long delta = entry.minValue; + return new DenseNumericDocValues(maxDoc) { + @Override + public long longValue() throws IOException { + return mul * values.get(doc) + delta; + } + }; + } + } + } + } else { + // sparse + final IndexedDISI disi = + new IndexedDISI( + data, + entry.docsWithFieldOffset, + entry.docsWithFieldLength, + entry.jumpTableEntryCount, + entry.denseRankPower, + entry.numValues); + if (entry.bitsPerValue == 0) { + return new SparseNumericDocValues(disi) { + @Override + public long longValue() throws IOException { + return entry.minValue; + } + }; + } else { + final RandomAccessInput slice = + data.randomAccessSlice(entry.valuesOffset, entry.valuesLength); + if (entry.blockShift >= 0) { + // sparse and split into blocks of different bits per value + return new SparseNumericDocValues(disi) { + final VaryingBPVReader vBPVReader = new VaryingBPVReader(entry, slice); + + @Override + public long longValue() throws IOException { + final int index = disi.index(); + return vBPVReader.getLongValue(index); + } + }; + } else { + final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue); + if (entry.table != null) { + final long[] table = entry.table; + return new SparseNumericDocValues(disi) { + @Override + public long longValue() throws IOException { + return table[(int) values.get(disi.index())]; + } + }; + } else { + final long mul = entry.gcd; + final long delta = entry.minValue; + return new SparseNumericDocValues(disi) { + @Override + public long longValue() throws IOException { + return mul * values.get(disi.index()) + delta; + } + }; + } + } + } + } + } + + private LongValues getNumericValues(NumericEntry entry) throws IOException { + if (entry.bitsPerValue == 0) { + return new LongValues() { + @Override + public long get(long index) { + return entry.minValue; + } + }; + } else { + final RandomAccessInput slice = + data.randomAccessSlice(entry.valuesOffset, entry.valuesLength); + if (entry.blockShift >= 0) { + return new LongValues() { + final VaryingBPVReader vBPVReader = new VaryingBPVReader(entry, slice); + + @Override + public long get(long index) { + try { + return vBPVReader.getLongValue(index); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + }; + } else { + final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue); + if (entry.table != null) { + final long[] table = entry.table; + return new LongValues() { + @Override + public long get(long index) { + return table[(int) values.get(index)]; + } + }; + } else if (entry.gcd != 1) { + final long gcd = entry.gcd; + final long minValue = entry.minValue; + return new LongValues() { + @Override + public long get(long index) { + return values.get(index) * gcd + minValue; + } + }; + } else if (entry.minValue != 0) { + final long minValue = entry.minValue; + return new LongValues() { + @Override + public long get(long index) { + return values.get(index) + minValue; + } + }; + } else { + return values; + } + } + } + } + + private abstract static class DenseBinaryDocValues extends BinaryDocValues { + + final int maxDoc; + int doc = -1; + + DenseBinaryDocValues(int maxDoc) { + this.maxDoc = maxDoc; + } + + @Override + public int nextDoc() throws IOException { + return advance(doc + 1); + } + + @Override + public int docID() { + return doc; + } + + @Override + public long cost() { + return maxDoc; + } + + @Override + public int advance(int target) throws IOException { + if (target >= maxDoc) { + return doc = NO_MORE_DOCS; + } + return doc = target; + } + + @Override + public boolean advanceExact(int target) throws IOException { + doc = target; + return true; + } + } + + private abstract static class SparseBinaryDocValues extends BinaryDocValues { + + final IndexedDISI disi; + + SparseBinaryDocValues(IndexedDISI disi) { + this.disi = disi; + } + + @Override + public int nextDoc() throws IOException { + return disi.nextDoc(); + } + + @Override + public int docID() { + return disi.docID(); + } + + @Override + public long cost() { + return disi.cost(); + } + + @Override + public int advance(int target) throws IOException { + return disi.advance(target); + } + + @Override + public boolean advanceExact(int target) throws IOException { + return disi.advanceExact(target); + } + } + + private BinaryDocValues getUncompressedBinary(BinaryEntry entry) throws IOException { + if (entry.docsWithFieldOffset == -2) { + return DocValues.emptyBinary(); + } + + final IndexInput bytesSlice = data.slice("fixed-binary", entry.dataOffset, entry.dataLength); + + if (entry.docsWithFieldOffset == -1) { + // dense + if (entry.minLength == entry.maxLength) { + // fixed length + final int length = entry.maxLength; + return new DenseBinaryDocValues(maxDoc) { + final BytesRef bytes = new BytesRef(new byte[length], 0, length); + + @Override + public BytesRef binaryValue() throws IOException { + bytesSlice.seek((long) doc * length); + bytesSlice.readBytes(bytes.bytes, 0, length); + return bytes; + } + }; + } else { + // variable length + final RandomAccessInput addressesData = + this.data.randomAccessSlice(entry.addressesOffset, entry.addressesLength); + final LongValues addresses = + DirectMonotonicReader.getInstance(entry.addressesMeta, addressesData); + return new DenseBinaryDocValues(maxDoc) { + final BytesRef bytes = new BytesRef(new byte[entry.maxLength], 0, entry.maxLength); + + @Override + public BytesRef binaryValue() throws IOException { + long startOffset = addresses.get(doc); + bytes.length = (int) (addresses.get(doc + 1L) - startOffset); + bytesSlice.seek(startOffset); + bytesSlice.readBytes(bytes.bytes, 0, bytes.length); + return bytes; + } + }; + } + } else { + // sparse + final IndexedDISI disi = + new IndexedDISI( + data, + entry.docsWithFieldOffset, + entry.docsWithFieldLength, + entry.jumpTableEntryCount, + entry.denseRankPower, + entry.numDocsWithField); + if (entry.minLength == entry.maxLength) { + // fixed length + final int length = entry.maxLength; + return new SparseBinaryDocValues(disi) { + final BytesRef bytes = new BytesRef(new byte[length], 0, length); + + @Override + public BytesRef binaryValue() throws IOException { + bytesSlice.seek((long) disi.index() * length); + bytesSlice.readBytes(bytes.bytes, 0, length); + return bytes; + } + }; + } else { + // variable length + final RandomAccessInput addressesData = + this.data.randomAccessSlice(entry.addressesOffset, entry.addressesLength); + final LongValues addresses = + DirectMonotonicReader.getInstance(entry.addressesMeta, addressesData); + return new SparseBinaryDocValues(disi) { + final BytesRef bytes = new BytesRef(new byte[entry.maxLength], 0, entry.maxLength); + + @Override + public BytesRef binaryValue() throws IOException { + final int index = disi.index(); + long startOffset = addresses.get(index); + bytes.length = (int) (addresses.get(index + 1L) - startOffset); + bytesSlice.seek(startOffset); + bytesSlice.readBytes(bytes.bytes, 0, bytes.length); + return bytes; + } + }; + } + } + } + + // Decompresses blocks of binary values to retrieve content + static class BinaryDecoder { + + private final LongValues addresses; + private final IndexInput compressedData; + // Cache of last uncompressed block + private long lastBlockId = -1; + private final int[] uncompressedDocStarts; + private int uncompressedBlockLength = 0; + private final byte[] uncompressedBlock; + private final BytesRef uncompressedBytesRef; + private final int docsPerChunk; + private final int docsPerChunkShift; + + public BinaryDecoder( + LongValues addresses, + IndexInput compressedData, + int biggestUncompressedBlockSize, + int docsPerChunkShift) { + super(); + this.addresses = addresses; + this.compressedData = compressedData; + // pre-allocate a byte array large enough for the biggest uncompressed block needed. + this.uncompressedBlock = new byte[biggestUncompressedBlockSize]; + uncompressedBytesRef = new BytesRef(uncompressedBlock); + this.docsPerChunk = 1 << docsPerChunkShift; + this.docsPerChunkShift = docsPerChunkShift; + uncompressedDocStarts = new int[docsPerChunk + 1]; + } + + BytesRef decode(int docNumber) throws IOException { + int blockId = docNumber >> docsPerChunkShift; + int docInBlockId = docNumber % docsPerChunk; + assert docInBlockId < docsPerChunk; + + // already read and uncompressed? + if (blockId != lastBlockId) { + lastBlockId = blockId; + long blockStartOffset = addresses.get(blockId); + compressedData.seek(blockStartOffset); + + uncompressedBlockLength = 0; + + int onlyLength = -1; + for (int i = 0; i < docsPerChunk; i++) { + if (i == 0) { + // The first length value is special. It is shifted and has a bit to denote if + // all other values are the same length + int lengthPlusSameInd = compressedData.readVInt(); + int sameIndicator = lengthPlusSameInd & 1; + int firstValLength = lengthPlusSameInd >>> 1; + if (sameIndicator == 1) { + onlyLength = firstValLength; + } + uncompressedBlockLength += firstValLength; + } else { + if (onlyLength == -1) { + // Various lengths are stored - read each from disk + uncompressedBlockLength += compressedData.readVInt(); + } else { + // Only one length + uncompressedBlockLength += onlyLength; + } + } + uncompressedDocStarts[i + 1] = uncompressedBlockLength; + } + + if (uncompressedBlockLength == 0) { + uncompressedBytesRef.offset = 0; + uncompressedBytesRef.length = 0; + return uncompressedBytesRef; + } + + assert uncompressedBlockLength <= uncompressedBlock.length; + LZ4.decompress(compressedData, uncompressedBlockLength, uncompressedBlock, 0); + } + + uncompressedBytesRef.offset = uncompressedDocStarts[docInBlockId]; + uncompressedBytesRef.length = + uncompressedDocStarts[docInBlockId + 1] - uncompressedBytesRef.offset; + return uncompressedBytesRef; + } + } + + @Override + public BinaryDocValues getBinary(FieldInfo field) throws IOException { + BinaryEntry entry = binaries.get(field.name); + if (entry.compressed) { + return getCompressedBinary(entry); + } else { + return getUncompressedBinary(entry); + } + } + + private BinaryDocValues getCompressedBinary(BinaryEntry entry) throws IOException { + + if (entry.docsWithFieldOffset == -2) { + return DocValues.emptyBinary(); + } + if (entry.docsWithFieldOffset == -1) { + // dense + final RandomAccessInput addressesData = + this.data.randomAccessSlice(entry.addressesOffset, entry.addressesLength); + final LongValues addresses = + DirectMonotonicReader.getInstance(entry.addressesMeta, addressesData); + return new DenseBinaryDocValues(maxDoc) { + BinaryDecoder decoder = + new BinaryDecoder( + addresses, data.clone(), entry.maxUncompressedChunkSize, entry.docsPerChunkShift); + + @Override + public BytesRef binaryValue() throws IOException { + return decoder.decode(doc); + } + }; + } else { + // sparse + final IndexedDISI disi = + new IndexedDISI( + data, + entry.docsWithFieldOffset, + entry.docsWithFieldLength, + entry.jumpTableEntryCount, + entry.denseRankPower, + entry.numDocsWithField); + final RandomAccessInput addressesData = + this.data.randomAccessSlice(entry.addressesOffset, entry.addressesLength); + final LongValues addresses = + DirectMonotonicReader.getInstance(entry.addressesMeta, addressesData); + return new SparseBinaryDocValues(disi) { + BinaryDecoder decoder = + new BinaryDecoder( + addresses, data.clone(), entry.maxUncompressedChunkSize, entry.docsPerChunkShift); + + @Override + public BytesRef binaryValue() throws IOException { + return decoder.decode(disi.index()); + } + }; + } + } + + @Override + public SortedDocValues getSorted(FieldInfo field) throws IOException { + SortedEntry entry = sorted.get(field.name); + return getSorted(entry); + } + + private SortedDocValues getSorted(SortedEntry entry) throws IOException { + if (entry.docsWithFieldOffset == -2) { + return DocValues.emptySorted(); + } + + final LongValues ords; + if (entry.bitsPerValue == 0) { + ords = + new LongValues() { + @Override + public long get(long index) { + return 0L; + } + }; + } else { + final RandomAccessInput slice = data.randomAccessSlice(entry.ordsOffset, entry.ordsLength); + ords = DirectReader.getInstance(slice, entry.bitsPerValue); + } + + if (entry.docsWithFieldOffset == -1) { + // dense + return new BaseSortedDocValues(entry, data) { + + int doc = -1; + + @Override + public int nextDoc() throws IOException { + return advance(doc + 1); + } + + @Override + public int docID() { + return doc; + } + + @Override + public long cost() { + return maxDoc; + } + + @Override + public int advance(int target) throws IOException { + if (target >= maxDoc) { + return doc = NO_MORE_DOCS; + } + return doc = target; + } + + @Override + public boolean advanceExact(int target) { + doc = target; + return true; + } + + @Override + public int ordValue() { + return (int) ords.get(doc); + } + }; + } else { + // sparse + final IndexedDISI disi = + new IndexedDISI( + data, + entry.docsWithFieldOffset, + entry.docsWithFieldLength, + entry.jumpTableEntryCount, + entry.denseRankPower, + entry.numDocsWithField); + return new BaseSortedDocValues(entry, data) { + + @Override + public int nextDoc() throws IOException { + return disi.nextDoc(); + } + + @Override + public int docID() { + return disi.docID(); + } + + @Override + public long cost() { + return disi.cost(); + } + + @Override + public int advance(int target) throws IOException { + return disi.advance(target); + } + + @Override + public boolean advanceExact(int target) throws IOException { + return disi.advanceExact(target); + } + + @Override + public int ordValue() { + return (int) ords.get(disi.index()); + } + }; + } + } + + private abstract static class BaseSortedDocValues extends SortedDocValues { + + final SortedEntry entry; + final IndexInput data; + final TermsEnum termsEnum; + + BaseSortedDocValues(SortedEntry entry, IndexInput data) throws IOException { + this.entry = entry; + this.data = data; + this.termsEnum = termsEnum(); + } + + @Override + public int getValueCount() { + return Math.toIntExact(entry.termsDictSize); + } + + @Override + public BytesRef lookupOrd(int ord) throws IOException { + termsEnum.seekExact(ord); + return termsEnum.term(); + } + + @Override + public int lookupTerm(BytesRef key) throws IOException { + SeekStatus status = termsEnum.seekCeil(key); + switch (status) { + case FOUND: + return Math.toIntExact(termsEnum.ord()); + default: + return Math.toIntExact(-1L - termsEnum.ord()); + } + } + + @Override + public TermsEnum termsEnum() throws IOException { + return new TermsDict(entry, data); + } + } + + private abstract static class BaseSortedSetDocValues extends SortedSetDocValues { + + final SortedSetEntry entry; + final IndexInput data; + final TermsEnum termsEnum; + + BaseSortedSetDocValues(SortedSetEntry entry, IndexInput data) throws IOException { + this.entry = entry; + this.data = data; + this.termsEnum = termsEnum(); + } + + @Override + public long getValueCount() { + return entry.termsDictSize; + } + + @Override + public BytesRef lookupOrd(long ord) throws IOException { + termsEnum.seekExact(ord); + return termsEnum.term(); + } + + @Override + public long lookupTerm(BytesRef key) throws IOException { + SeekStatus status = termsEnum.seekCeil(key); + switch (status) { + case FOUND: + return termsEnum.ord(); + default: + return -1L - termsEnum.ord(); + } + } + + @Override + public TermsEnum termsEnum() throws IOException { + return new TermsDict(entry, data); + } + } + + private static class TermsDict extends BaseTermsEnum { + static final int LZ4_DECOMPRESSOR_PADDING = 7; + + final TermsDictEntry entry; + final LongValues blockAddresses; + final IndexInput bytes; + final long blockMask; + final LongValues indexAddresses; + final IndexInput indexBytes; + final BytesRef term; + long ord = -1; + + BytesRef blockBuffer = null; + ByteArrayDataInput blockInput = null; + long currentCompressedBlockStart = -1; + long currentCompressedBlockEnd = -1; + + TermsDict(TermsDictEntry entry, IndexInput data) throws IOException { + this.entry = entry; + RandomAccessInput addressesSlice = + data.randomAccessSlice(entry.termsAddressesOffset, entry.termsAddressesLength); + blockAddresses = DirectMonotonicReader.getInstance(entry.termsAddressesMeta, addressesSlice); + bytes = data.slice("terms", entry.termsDataOffset, entry.termsDataLength); + blockMask = (1L << entry.termsDictBlockShift) - 1; + RandomAccessInput indexAddressesSlice = + data.randomAccessSlice(entry.termsIndexAddressesOffset, entry.termsIndexAddressesLength); + indexAddresses = + DirectMonotonicReader.getInstance(entry.termsIndexAddressesMeta, indexAddressesSlice); + indexBytes = data.slice("terms-index", entry.termsIndexOffset, entry.termsIndexLength); + term = new BytesRef(entry.maxTermLength); + + if (entry.compressed) { + // add 7 padding bytes can help decompression run faster. + int bufferSize = entry.maxBlockLength + LZ4_DECOMPRESSOR_PADDING; + blockBuffer = new BytesRef(new byte[bufferSize], 0, bufferSize); + } + } + + @Override + public BytesRef next() throws IOException { + if (++ord >= entry.termsDictSize) { + return null; + } + + if ((ord & blockMask) == 0L) { + if (this.entry.compressed) { + decompressBlock(); + } else { + term.length = bytes.readVInt(); + bytes.readBytes(term.bytes, 0, term.length); + } + } else { + DataInput input = this.entry.compressed ? blockInput : bytes; + final int token = Byte.toUnsignedInt(input.readByte()); + int prefixLength = token & 0x0F; + int suffixLength = 1 + (token >>> 4); + if (prefixLength == 15) { + prefixLength += input.readVInt(); + } + if (suffixLength == 16) { + suffixLength += input.readVInt(); + } + term.length = prefixLength + suffixLength; + input.readBytes(term.bytes, prefixLength, suffixLength); + } + return term; + } + + @Override + public void seekExact(long ord) throws IOException { + if (ord < 0 || ord >= entry.termsDictSize) { + throw new IndexOutOfBoundsException(); + } + final long blockIndex = ord >>> entry.termsDictBlockShift; + final long blockAddress = blockAddresses.get(blockIndex); + bytes.seek(blockAddress); + this.ord = (blockIndex << entry.termsDictBlockShift) - 1; + do { + next(); + } while (this.ord < ord); + } + + private BytesRef getTermFromIndex(long index) throws IOException { + assert index >= 0 && index <= (entry.termsDictSize - 1) >>> entry.termsDictIndexShift; + final long start = indexAddresses.get(index); + term.length = (int) (indexAddresses.get(index + 1) - start); + indexBytes.seek(start); + indexBytes.readBytes(term.bytes, 0, term.length); + return term; + } + + private long seekTermsIndex(BytesRef text) throws IOException { + long lo = 0L; + long hi = (entry.termsDictSize - 1) >>> entry.termsDictIndexShift; + while (lo <= hi) { + final long mid = (lo + hi) >>> 1; + getTermFromIndex(mid); + final int cmp = term.compareTo(text); + if (cmp <= 0) { + lo = mid + 1; + } else { + hi = mid - 1; + } + } + + assert hi < 0 || getTermFromIndex(hi).compareTo(text) <= 0; + assert hi == ((entry.termsDictSize - 1) >>> entry.termsDictIndexShift) + || getTermFromIndex(hi + 1).compareTo(text) > 0; + + return hi; + } + + private BytesRef getFirstTermFromBlock(long block) throws IOException { + assert block >= 0 && block <= (entry.termsDictSize - 1) >>> entry.termsDictBlockShift; + final long blockAddress = blockAddresses.get(block); + bytes.seek(blockAddress); + term.length = bytes.readVInt(); + bytes.readBytes(term.bytes, 0, term.length); + return term; + } + + private long seekBlock(BytesRef text) throws IOException { + long index = seekTermsIndex(text); + if (index == -1L) { + return -1L; + } + + long ordLo = index << entry.termsDictIndexShift; + long ordHi = Math.min(entry.termsDictSize, ordLo + (1L << entry.termsDictIndexShift)) - 1L; + + long blockLo = ordLo >>> entry.termsDictBlockShift; + long blockHi = ordHi >>> entry.termsDictBlockShift; + + while (blockLo <= blockHi) { + final long blockMid = (blockLo + blockHi) >>> 1; + getFirstTermFromBlock(blockMid); + final int cmp = term.compareTo(text); + if (cmp <= 0) { + blockLo = blockMid + 1; + } else { + blockHi = blockMid - 1; + } + } + + assert blockHi < 0 || getFirstTermFromBlock(blockHi).compareTo(text) <= 0; + assert blockHi == ((entry.termsDictSize - 1) >>> entry.termsDictBlockShift) + || getFirstTermFromBlock(blockHi + 1).compareTo(text) > 0; + + return blockHi; + } + + @Override + public SeekStatus seekCeil(BytesRef text) throws IOException { + final long block = seekBlock(text); + if (block == -1) { + // before the first term + seekExact(0L); + return SeekStatus.NOT_FOUND; + } + final long blockAddress = blockAddresses.get(block); + this.ord = block << entry.termsDictBlockShift; + bytes.seek(blockAddress); + if (this.entry.compressed) { + decompressBlock(); + } else { + term.length = bytes.readVInt(); + bytes.readBytes(term.bytes, 0, term.length); + } + + while (true) { + int cmp = term.compareTo(text); + if (cmp == 0) { + return SeekStatus.FOUND; + } else if (cmp > 0) { + return SeekStatus.NOT_FOUND; + } + if (next() == null) { + return SeekStatus.END; + } + } + } + + private void decompressBlock() throws IOException { + // The first term is kept uncompressed, so no need to decompress block if only + // look up the first term when doing seek block. + term.length = bytes.readVInt(); + bytes.readBytes(term.bytes, 0, term.length); + long offset = bytes.getFilePointer(); + if (offset < entry.termsDataLength - 1) { + // Avoid decompress again if we are reading a same block. + if (currentCompressedBlockStart != offset) { + int decompressLength = bytes.readVInt(); + // Decompress the remaining of current block + LZ4.decompress(bytes, decompressLength, blockBuffer.bytes, 0); + currentCompressedBlockStart = offset; + currentCompressedBlockEnd = bytes.getFilePointer(); + } else { + // Skip decompression but need to re-seek to block end. + bytes.seek(currentCompressedBlockEnd); + } + + // Reset the buffer. + blockInput = new ByteArrayDataInput(blockBuffer.bytes, 0, blockBuffer.length); + } + } + + @Override + public BytesRef term() throws IOException { + return term; + } + + @Override + public long ord() throws IOException { + return ord; + } + + @Override + public long totalTermFreq() throws IOException { + return -1L; + } + + @Override + public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public ImpactsEnum impacts(int flags) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public int docFreq() throws IOException { + throw new UnsupportedOperationException(); + } + } + + @Override + public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException { + SortedNumericEntry entry = sortedNumerics.get(field.name); + if (entry.numValues == entry.numDocsWithField) { + return DocValues.singleton(getNumeric(entry)); + } + + final RandomAccessInput addressesInput = + data.randomAccessSlice(entry.addressesOffset, entry.addressesLength); + final LongValues addresses = + DirectMonotonicReader.getInstance(entry.addressesMeta, addressesInput); + + final LongValues values = getNumericValues(entry); + + if (entry.docsWithFieldOffset == -1) { + // dense + return new SortedNumericDocValues() { + + int doc = -1; + long start, end; + int count; + + @Override + public int nextDoc() throws IOException { + return advance(doc + 1); + } + + @Override + public int docID() { + return doc; + } + + @Override + public long cost() { + return maxDoc; + } + + @Override + public int advance(int target) throws IOException { + if (target >= maxDoc) { + return doc = NO_MORE_DOCS; + } + start = addresses.get(target); + end = addresses.get(target + 1L); + count = (int) (end - start); + return doc = target; + } + + @Override + public boolean advanceExact(int target) throws IOException { + start = addresses.get(target); + end = addresses.get(target + 1L); + count = (int) (end - start); + doc = target; + return true; + } + + @Override + public long nextValue() throws IOException { + return values.get(start++); + } + + @Override + public int docValueCount() { + return count; + } + }; + } else { + // sparse + final IndexedDISI disi = + new IndexedDISI( + data, + entry.docsWithFieldOffset, + entry.docsWithFieldLength, + entry.jumpTableEntryCount, + entry.denseRankPower, + entry.numDocsWithField); + return new SortedNumericDocValues() { + + boolean set; + long start, end; + int count; + + @Override + public int nextDoc() throws IOException { + set = false; + return disi.nextDoc(); + } + + @Override + public int docID() { + return disi.docID(); + } + + @Override + public long cost() { + return disi.cost(); + } + + @Override + public int advance(int target) throws IOException { + set = false; + return disi.advance(target); + } + + @Override + public boolean advanceExact(int target) throws IOException { + set = false; + return disi.advanceExact(target); + } + + @Override + public long nextValue() throws IOException { + set(); + return values.get(start++); + } + + @Override + public int docValueCount() { + set(); + return count; + } + + private void set() { + if (set == false) { + final int index = disi.index(); + start = addresses.get(index); + end = addresses.get(index + 1L); + count = (int) (end - start); + set = true; + } + } + }; + } + } + + @Override + public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException { + SortedSetEntry entry = sortedSets.get(field.name); + if (entry.singleValueEntry != null) { + return DocValues.singleton(getSorted(entry.singleValueEntry)); + } + + final RandomAccessInput slice = data.randomAccessSlice(entry.ordsOffset, entry.ordsLength); + final LongValues ords = DirectReader.getInstance(slice, entry.bitsPerValue); + + final RandomAccessInput addressesInput = + data.randomAccessSlice(entry.addressesOffset, entry.addressesLength); + final LongValues addresses = + DirectMonotonicReader.getInstance(entry.addressesMeta, addressesInput); + + if (entry.docsWithFieldOffset == -1) { + // dense + return new BaseSortedSetDocValues(entry, data) { + + int doc = -1; + long start; + long end; + + @Override + public int nextDoc() throws IOException { + return advance(doc + 1); + } + + @Override + public int docID() { + return doc; + } + + @Override + public long cost() { + return maxDoc; + } + + @Override + public int advance(int target) throws IOException { + if (target >= maxDoc) { + return doc = NO_MORE_DOCS; + } + start = addresses.get(target); + end = addresses.get(target + 1L); + return doc = target; + } + + @Override + public boolean advanceExact(int target) throws IOException { + start = addresses.get(target); + end = addresses.get(target + 1L); + doc = target; + return true; + } + + @Override + public long nextOrd() throws IOException { + if (start == end) { + return NO_MORE_ORDS; + } + return ords.get(start++); + } + }; + } else { + // sparse + final IndexedDISI disi = + new IndexedDISI( + data, + entry.docsWithFieldOffset, + entry.docsWithFieldLength, + entry.jumpTableEntryCount, + entry.denseRankPower, + entry.numDocsWithField); + return new BaseSortedSetDocValues(entry, data) { + + boolean set; + long start; + long end = 0; + + @Override + public int nextDoc() throws IOException { + set = false; + return disi.nextDoc(); + } + + @Override + public int docID() { + return disi.docID(); + } + + @Override + public long cost() { + return disi.cost(); + } + + @Override + public int advance(int target) throws IOException { + set = false; + return disi.advance(target); + } + + @Override + public boolean advanceExact(int target) throws IOException { + set = false; + return disi.advanceExact(target); + } + + @Override + public long nextOrd() throws IOException { + if (set == false) { + final int index = disi.index(); + final long start = addresses.get(index); + this.start = start + 1; + end = addresses.get(index + 1L); + set = true; + return ords.get(start); + } else if (start == end) { + return NO_MORE_ORDS; + } else { + return ords.get(start++); + } + } + }; + } + } + + @Override + public void checkIntegrity() throws IOException { + CodecUtil.checksumEntireFile(data); + } + + /** + * Reader for longs split into blocks of different bits per values. The longs are requested by + * index and must be accessed in monotonically increasing order. + */ + // Note: The order requirement could be removed as the jump-tables allow for backwards iteration + // Note 2: The rankSlice is only used if an advance of > 1 block is called. Its construction could + // be lazy + private class VaryingBPVReader { + final RandomAccessInput slice; // 2 slices to avoid cache thrashing when using rank + final RandomAccessInput rankSlice; + final NumericEntry entry; + final int shift; + final long mul; + final int mask; + + long block = -1; + long delta; + long offset; + long blockEndOffset; + LongValues values; + + VaryingBPVReader(NumericEntry entry, RandomAccessInput slice) throws IOException { + this.entry = entry; + this.slice = slice; + this.rankSlice = + entry.valueJumpTableOffset == -1 + ? null + : data.randomAccessSlice( + entry.valueJumpTableOffset, data.length() - entry.valueJumpTableOffset); + shift = entry.blockShift; + mul = entry.gcd; + mask = (1 << shift) - 1; + } + + long getLongValue(long index) throws IOException { + final long block = index >>> shift; + if (this.block != block) { + int bitsPerValue; + do { + // If the needed block is the one directly following the current block, it is cheaper to + // avoid the cache + if (rankSlice != null && block != this.block + 1) { + blockEndOffset = rankSlice.readLong(block * Long.BYTES) - entry.valuesOffset; + this.block = block - 1; + } + offset = blockEndOffset; + bitsPerValue = slice.readByte(offset++); + delta = slice.readLong(offset); + offset += Long.BYTES; + if (bitsPerValue == 0) { + blockEndOffset = offset; + } else { + final int length = slice.readInt(offset); + offset += Integer.BYTES; + blockEndOffset = offset + length; + } + this.block++; + } while (this.block != block); + values = + bitsPerValue == 0 + ? LongValues.ZEROES + : DirectReader.getInstance(slice, bitsPerValue, offset); + } + return mul * values.get(index & mask) + delta; + } + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90NormsConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90NormsConsumer.java new file mode 100644 index 00000000000..5157cabcc75 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90NormsConsumer.java @@ -0,0 +1,176 @@ +/* + * 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 static org.apache.lucene.codecs.lucene90.Lucene90NormsFormat.VERSION_CURRENT; + +import java.io.IOException; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.NormsConsumer; +import org.apache.lucene.codecs.NormsProducer; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.NumericDocValues; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.IOUtils; + +/** Writer for {@link Lucene90NormsFormat} */ +final class Lucene90NormsConsumer extends NormsConsumer { + IndexOutput data, meta; + final int maxDoc; + + Lucene90NormsConsumer( + SegmentWriteState state, + String dataCodec, + String dataExtension, + String metaCodec, + String metaExtension) + throws IOException { + boolean success = false; + try { + String dataName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, state.segmentSuffix, dataExtension); + data = state.directory.createOutput(dataName, state.context); + CodecUtil.writeIndexHeader( + data, dataCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); + String metaName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, state.segmentSuffix, metaExtension); + meta = state.directory.createOutput(metaName, state.context); + CodecUtil.writeIndexHeader( + meta, metaCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); + maxDoc = state.segmentInfo.maxDoc(); + success = true; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(this); + } + } + } + + @Override + public void close() throws IOException { + boolean success = false; + try { + if (meta != null) { + meta.writeInt(-1); // write EOF marker + CodecUtil.writeFooter(meta); // write checksum + } + if (data != null) { + CodecUtil.writeFooter(data); // write checksum + } + success = true; + } finally { + if (success) { + IOUtils.close(data, meta); + } else { + IOUtils.closeWhileHandlingException(data, meta); + } + meta = data = null; + } + } + + @Override + public void addNormsField(FieldInfo field, NormsProducer normsProducer) throws IOException { + NumericDocValues values = normsProducer.getNorms(field); + int numDocsWithValue = 0; + long min = Long.MAX_VALUE; + long max = Long.MIN_VALUE; + for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) { + numDocsWithValue++; + long v = values.longValue(); + min = Math.min(min, v); + max = Math.max(max, v); + } + assert numDocsWithValue <= maxDoc; + + meta.writeInt(field.number); + + if (numDocsWithValue == 0) { + meta.writeLong(-2); // docsWithFieldOffset + meta.writeLong(0L); // docsWithFieldLength + meta.writeShort((short) -1); // jumpTableEntryCount + meta.writeByte((byte) -1); // denseRankPower + } else if (numDocsWithValue == maxDoc) { + meta.writeLong(-1); // docsWithFieldOffset + meta.writeLong(0L); // docsWithFieldLength + meta.writeShort((short) -1); // jumpTableEntryCount + meta.writeByte((byte) -1); // denseRankPower + } else { + long offset = data.getFilePointer(); + meta.writeLong(offset); // docsWithFieldOffset + values = normsProducer.getNorms(field); + final short jumpTableEntryCount = + IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER); + meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength + meta.writeShort(jumpTableEntryCount); + meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER); + } + + meta.writeInt(numDocsWithValue); + int numBytesPerValue = numBytesPerValue(min, max); + + meta.writeByte((byte) numBytesPerValue); + if (numBytesPerValue == 0) { + meta.writeLong(min); + } else { + meta.writeLong(data.getFilePointer()); // normsOffset + values = normsProducer.getNorms(field); + writeValues(values, numBytesPerValue, data); + } + } + + private int numBytesPerValue(long min, long max) { + if (min >= max) { + return 0; + } else if (min >= Byte.MIN_VALUE && max <= Byte.MAX_VALUE) { + return 1; + } else if (min >= Short.MIN_VALUE && max <= Short.MAX_VALUE) { + return 2; + } else if (min >= Integer.MIN_VALUE && max <= Integer.MAX_VALUE) { + return 4; + } else { + return 8; + } + } + + private void writeValues(NumericDocValues values, int numBytesPerValue, IndexOutput out) + throws IOException, AssertionError { + for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) { + long value = values.longValue(); + switch (numBytesPerValue) { + case 1: + out.writeByte((byte) value); + break; + case 2: + out.writeShort((short) value); + break; + case 4: + out.writeInt((int) value); + break; + case 8: + out.writeLong(value); + break; + default: + throw new AssertionError(); + } + } + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90NormsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90NormsFormat.java new file mode 100644 index 00000000000..b0d60c29f3c --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90NormsFormat.java @@ -0,0 +1,106 @@ +/* + * 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 org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.NormsConsumer; +import org.apache.lucene.codecs.NormsFormat; +import org.apache.lucene.codecs.NormsProducer; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.store.DataOutput; + +/** + * Lucene 9.0 Score normalization format. + * + *

Encodes normalization values by encoding each value with the minimum number of bytes needed to + * represent the range (which can be zero). + * + *

Files: + * + *

    + *
  1. .nvd: Norms data + *
  2. .nvm: Norms metadata + *
+ * + *
    + *
  1. + *

    The Norms metadata or .nvm file. + *

    For each norms field, this stores metadata, such as the offset into the Norms data + * (.nvd) + *

    Norms metadata (.dvm) --> Header,<Entry>NumFields,Footer + *

      + *
    • Header --> {@link CodecUtil#writeIndexHeader IndexHeader} + *
    • Entry --> FieldNumber, DocsWithFieldAddress, DocsWithFieldLength, + * NumDocsWithField, BytesPerNorm, NormsAddress + *
    • FieldNumber --> {@link DataOutput#writeInt Int32} + *
    • DocsWithFieldAddress --> {@link DataOutput#writeLong Int64} + *
    • DocsWithFieldLength --> {@link DataOutput#writeLong Int64} + *
    • NumDocsWithField --> {@link DataOutput#writeInt Int32} + *
    • BytesPerNorm --> {@link DataOutput#writeByte byte} + *
    • NormsAddress --> {@link DataOutput#writeLong Int64} + *
    • Footer --> {@link CodecUtil#writeFooter CodecFooter} + *
    + *

    FieldNumber of -1 indicates the end of metadata. + *

    NormsAddress is the pointer to the start of the data in the norms data (.nvd), or the + * singleton value when BytesPerValue = 0. If BytesPerValue is different from 0 then there are + * NumDocsWithField values to read at that offset. + *

    DocsWithFieldAddress is the pointer to the start of the bit set containing documents + * that have a norm in the norms data (.nvd), or -2 if no documents have a norm value, or -1 + * if all documents have a norm value. + *

    DocsWithFieldLength is the number of bytes used to encode the set of documents that have + * a norm. + *

  2. + *

    The Norms data or .nvd file. + *

    For each Norms field, this stores the actual per-document data (the heavy-lifting) + *

    Norms data (.nvd) --> Header,< Data >NumFields,Footer + *

      + *
    • Header --> {@link CodecUtil#writeIndexHeader IndexHeader} + *
    • DocsWithFieldData --> {@link IndexedDISI#writeBitSet Bit set of MaxDoc bits} + *
    • NormsData --> {@link DataOutput#writeByte(byte) byte}NumDocsWithField * + * BytesPerValue + *
    • Footer --> {@link CodecUtil#writeFooter CodecFooter} + *
    + *
+ * + * @lucene.experimental + */ +public class Lucene90NormsFormat extends NormsFormat { + + /** Sole Constructor */ + public Lucene90NormsFormat() {} + + @Override + public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException { + return new Lucene90NormsConsumer( + state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION); + } + + @Override + public NormsProducer normsProducer(SegmentReadState state) throws IOException { + return new Lucene90NormsProducer( + state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION); + } + + private static final String DATA_CODEC = "Lucene90NormsData"; + private static final String DATA_EXTENSION = "nvd"; + private static final String METADATA_CODEC = "Lucene90NormsMetadata"; + private static final String METADATA_EXTENSION = "nvm"; + static final int VERSION_START = 0; + static final int VERSION_CURRENT = VERSION_START; +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90NormsProducer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90NormsProducer.java new file mode 100644 index 00000000000..fdeb0a7aa18 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90NormsProducer.java @@ -0,0 +1,492 @@ +/* + * 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 static org.apache.lucene.codecs.lucene90.Lucene90NormsFormat.VERSION_CURRENT; +import static org.apache.lucene.codecs.lucene90.Lucene90NormsFormat.VERSION_START; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.NormsProducer; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.index.DocValues; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.NumericDocValues; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.store.ChecksumIndexInput; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.RandomAccessInput; +import org.apache.lucene.util.IOUtils; + +/** Reader for {@link Lucene90NormsFormat} */ +final class Lucene90NormsProducer extends NormsProducer implements Cloneable { + // metadata maps (just file pointers and minimal stuff) + private final Map norms = new HashMap<>(); + private final int maxDoc; + private IndexInput data; + private boolean merging; + private Map disiInputs; + private Map disiJumpTables; + private Map dataInputs; + + Lucene90NormsProducer( + SegmentReadState state, + String dataCodec, + String dataExtension, + String metaCodec, + String metaExtension) + throws IOException { + maxDoc = state.segmentInfo.maxDoc(); + String metaName = + IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension); + int version = -1; + + // read in the entries from the metadata file. + try (ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context)) { + Throwable priorE = null; + try { + version = + CodecUtil.checkIndexHeader( + in, + metaCodec, + VERSION_START, + VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + readFields(in, state.fieldInfos); + } catch (Throwable exception) { + priorE = exception; + } finally { + CodecUtil.checkFooter(in, priorE); + } + } + + String dataName = + IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension); + data = state.directory.openInput(dataName, state.context); + boolean success = false; + try { + final int version2 = + CodecUtil.checkIndexHeader( + data, + dataCodec, + VERSION_START, + VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + if (version != version2) { + throw new CorruptIndexException( + "Format versions mismatch: meta=" + version + ",data=" + version2, data); + } + + // 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(data); + + success = true; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(this.data); + } + } + } + + @Override + public NormsProducer getMergeInstance() { + Lucene90NormsProducer clone; + try { + clone = (Lucene90NormsProducer) super.clone(); + } catch (CloneNotSupportedException e) { + // cannot happen + throw new RuntimeException(e); + } + clone.data = data.clone(); + clone.disiInputs = new HashMap<>(); + clone.disiJumpTables = new HashMap<>(); + clone.dataInputs = new HashMap<>(); + clone.merging = true; + return clone; + } + + static class NormsEntry { + byte denseRankPower; + byte bytesPerNorm; + long docsWithFieldOffset; + long docsWithFieldLength; + short jumpTableEntryCount; + int numDocsWithField; + long normsOffset; + } + + abstract static class DenseNormsIterator extends NumericDocValues { + + final int maxDoc; + int doc = -1; + + DenseNormsIterator(int maxDoc) { + this.maxDoc = maxDoc; + } + + @Override + public int docID() { + return doc; + } + + @Override + public int nextDoc() throws IOException { + return advance(doc + 1); + } + + @Override + public int advance(int target) throws IOException { + if (target >= maxDoc) { + return doc = NO_MORE_DOCS; + } + return doc = target; + } + + @Override + public boolean advanceExact(int target) throws IOException { + this.doc = target; + return true; + } + + @Override + public long cost() { + return maxDoc; + } + } + + abstract static class SparseNormsIterator extends NumericDocValues { + + final IndexedDISI disi; + + SparseNormsIterator(IndexedDISI disi) { + this.disi = disi; + } + + @Override + public int docID() { + return disi.docID(); + } + + @Override + public int nextDoc() throws IOException { + return disi.nextDoc(); + } + + @Override + public int advance(int target) throws IOException { + return disi.advance(target); + } + + @Override + public boolean advanceExact(int target) throws IOException { + return disi.advanceExact(target); + } + + @Override + public long cost() { + return disi.cost(); + } + } + + private void readFields(IndexInput meta, FieldInfos infos) throws IOException { + for (int fieldNumber = meta.readInt(); fieldNumber != -1; fieldNumber = meta.readInt()) { + FieldInfo info = infos.fieldInfo(fieldNumber); + if (info == null) { + throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta); + } else if (!info.hasNorms()) { + throw new CorruptIndexException("Invalid field: " + info.name, meta); + } + NormsEntry entry = new NormsEntry(); + entry.docsWithFieldOffset = meta.readLong(); + entry.docsWithFieldLength = meta.readLong(); + entry.jumpTableEntryCount = meta.readShort(); + entry.denseRankPower = meta.readByte(); + entry.numDocsWithField = meta.readInt(); + entry.bytesPerNorm = meta.readByte(); + switch (entry.bytesPerNorm) { + case 0: + case 1: + case 2: + case 4: + case 8: + break; + default: + throw new CorruptIndexException( + "Invalid bytesPerValue: " + entry.bytesPerNorm + ", field: " + info.name, meta); + } + entry.normsOffset = meta.readLong(); + norms.put(info.number, entry); + } + } + + private RandomAccessInput getDataInput(FieldInfo field, NormsEntry entry) throws IOException { + RandomAccessInput slice = null; + if (merging) { + slice = dataInputs.get(field.number); + } + if (slice == null) { + slice = + data.randomAccessSlice( + entry.normsOffset, entry.numDocsWithField * (long) entry.bytesPerNorm); + if (merging) { + dataInputs.put(field.number, slice); + } + } + return slice; + } + + private IndexInput getDisiInput(FieldInfo field, NormsEntry entry) throws IOException { + if (merging == false) { + return IndexedDISI.createBlockSlice( + data, + "docs", + entry.docsWithFieldOffset, + entry.docsWithFieldLength, + entry.jumpTableEntryCount); + } + + IndexInput in = disiInputs.get(field.number); + if (in == null) { + in = + IndexedDISI.createBlockSlice( + data, + "docs", + entry.docsWithFieldOffset, + entry.docsWithFieldLength, + entry.jumpTableEntryCount); + disiInputs.put(field.number, in); + } + + final IndexInput inF = in; // same as in but final + + // Wrap so that reads can be interleaved from the same thread if two + // norms instances are pulled and consumed in parallel. Merging usually + // doesn't need this feature but CheckIndex might, plus we need merge + // instances to behave well and not be trappy. + return new IndexInput("docs") { + + long offset = 0; + + @Override + public void readBytes(byte[] b, int off, int len) throws IOException { + inF.seek(offset); + offset += len; + inF.readBytes(b, off, len); + } + + @Override + public byte readByte() throws IOException { + throw new UnsupportedOperationException("Unused by IndexedDISI"); + } + + @Override + public IndexInput slice(String sliceDescription, long offset, long length) + throws IOException { + throw new UnsupportedOperationException("Unused by IndexedDISI"); + } + + @Override + public short readShort() throws IOException { + inF.seek(offset); + offset += Short.BYTES; + return inF.readShort(); + } + + @Override + public long readLong() throws IOException { + inF.seek(offset); + offset += Long.BYTES; + return inF.readLong(); + } + + @Override + public void seek(long pos) throws IOException { + offset = pos; + } + + @Override + public long length() { + throw new UnsupportedOperationException("Unused by IndexedDISI"); + } + + @Override + public long getFilePointer() { + return offset; + } + + @Override + public void close() throws IOException { + throw new UnsupportedOperationException("Unused by IndexedDISI"); + } + }; + } + + private RandomAccessInput getDisiJumpTable(FieldInfo field, NormsEntry entry) throws IOException { + RandomAccessInput jumpTable = null; + if (merging) { + jumpTable = disiJumpTables.get(field.number); + } + if (jumpTable == null) { + jumpTable = + IndexedDISI.createJumpTable( + data, + entry.docsWithFieldOffset, + entry.docsWithFieldLength, + entry.jumpTableEntryCount); + if (merging) { + disiJumpTables.put(field.number, jumpTable); + } + } + return jumpTable; + } + + @Override + public NumericDocValues getNorms(FieldInfo field) throws IOException { + final NormsEntry entry = norms.get(field.number); + if (entry.docsWithFieldOffset == -2) { + // empty + return DocValues.emptyNumeric(); + } else if (entry.docsWithFieldOffset == -1) { + // dense + if (entry.bytesPerNorm == 0) { + return new DenseNormsIterator(maxDoc) { + @Override + public long longValue() throws IOException { + return entry.normsOffset; + } + }; + } + final RandomAccessInput slice = getDataInput(field, entry); + switch (entry.bytesPerNorm) { + case 1: + return new DenseNormsIterator(maxDoc) { + @Override + public long longValue() throws IOException { + return slice.readByte(doc); + } + }; + case 2: + return new DenseNormsIterator(maxDoc) { + @Override + public long longValue() throws IOException { + return slice.readShort(((long) doc) << 1); + } + }; + case 4: + return new DenseNormsIterator(maxDoc) { + @Override + public long longValue() throws IOException { + return slice.readInt(((long) doc) << 2); + } + }; + case 8: + return new DenseNormsIterator(maxDoc) { + @Override + public long longValue() throws IOException { + return slice.readLong(((long) doc) << 3); + } + }; + default: + // should not happen, we already validate bytesPerNorm in readFields + throw new AssertionError(); + } + } else { + // sparse + final IndexInput disiInput = getDisiInput(field, entry); + final RandomAccessInput disiJumpTable = getDisiJumpTable(field, entry); + final IndexedDISI disi = + new IndexedDISI( + disiInput, + disiJumpTable, + entry.jumpTableEntryCount, + entry.denseRankPower, + entry.numDocsWithField); + + if (entry.bytesPerNorm == 0) { + return new SparseNormsIterator(disi) { + @Override + public long longValue() throws IOException { + return entry.normsOffset; + } + }; + } + final RandomAccessInput slice = getDataInput(field, entry); + switch (entry.bytesPerNorm) { + case 1: + return new SparseNormsIterator(disi) { + @Override + public long longValue() throws IOException { + return slice.readByte(disi.index()); + } + }; + case 2: + return new SparseNormsIterator(disi) { + @Override + public long longValue() throws IOException { + return slice.readShort(((long) disi.index()) << 1); + } + }; + case 4: + return new SparseNormsIterator(disi) { + @Override + public long longValue() throws IOException { + return slice.readInt(((long) disi.index()) << 2); + } + }; + case 8: + return new SparseNormsIterator(disi) { + @Override + public long longValue() throws IOException { + return slice.readLong(((long) disi.index()) << 3); + } + }; + default: + // should not happen, we already validate bytesPerNorm in readFields + throw new AssertionError(); + } + } + } + + @Override + public void close() throws IOException { + data.close(); + } + + @Override + public long ramBytesUsed() { + return 64L * norms.size(); // good enough + } + + @Override + public void checkIntegrity() throws IOException { + CodecUtil.checksumEntireFile(data); + } + + @Override + public String toString() { + return getClass().getSimpleName() + "(fields=" + norms.size() + ")"; + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/package-info.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/package-info.java index b615cf7ef7f..83a3912790d 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/package-info.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/package-info.java @@ -162,14 +162,14 @@ *
  • {@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat 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 + *
  • {@link org.apache.lucene.codecs.lucene90.Lucene90NormsFormat 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 + *
  • {@link org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat 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 @@ -280,12 +280,12 @@ * Stores additional per-position metadata information such as character offsets and user payloads * * - * {@link org.apache.lucene.codecs.lucene80.Lucene80NormsFormat Norms} + * {@link org.apache.lucene.codecs.lucene90.Lucene90NormsFormat Norms} * .nvd, .nvm * Encodes length and boost factors for docs and fields * * - * {@link org.apache.lucene.codecs.lucene80.Lucene80DocValuesFormat Per-Document Values} + * {@link org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat Per-Document Values} * .dvd, .dvm * Encodes additional scoring factors or other per-document information. * diff --git a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat index 43ed529ec01..5ef3a99f953 100644 --- a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat +++ b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat @@ -13,4 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.lucene.codecs.lucene80.Lucene80DocValuesFormat +org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestLucene90NormsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestLucene90NormsFormat.java new file mode 100644 index 00000000000..0b152d6ecf5 --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestLucene90NormsFormat.java @@ -0,0 +1,31 @@ +/* + * 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.lucene80; + +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.index.BaseNormsFormatTestCase; +import org.apache.lucene.util.TestUtil; + +/** Tests Lucene90NormsFormat */ +public class TestLucene90NormsFormat extends BaseNormsFormatTestCase { + private final Codec codec = TestUtil.getDefaultCodec(); + + @Override + protected Codec getCodec() { + return codec; + } +} diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestLucene90NormsFormatMergeInstance.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestLucene90NormsFormatMergeInstance.java new file mode 100644 index 00000000000..c9ba03f8bbb --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestLucene90NormsFormatMergeInstance.java @@ -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.lucene80; + +/** Test the merge instance of the Lucene90 norms format. */ +public class TestLucene90NormsFormatMergeInstance extends TestLucene90NormsFormat { + + @Override + protected boolean shouldTestMergeInstance() { + return true; + } +} diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/BaseLucene90DocValuesFormatTestCase.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/BaseLucene90DocValuesFormatTestCase.java new file mode 100644 index 00000000000..19bc5425ec9 --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/BaseLucene90DocValuesFormatTestCase.java @@ -0,0 +1,764 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.lucene.codecs.lucene90; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.TreeSet; +import java.util.function.LongSupplier; +import java.util.function.Supplier; +import org.apache.lucene.analysis.MockAnalyzer; +import org.apache.lucene.codecs.DocValuesFormat; +import org.apache.lucene.codecs.PostingsFormat; +import org.apache.lucene.codecs.asserting.AssertingCodec; +import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat; +import org.apache.lucene.document.BinaryDocValuesField; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.NumericDocValuesField; +import org.apache.lucene.document.SortedDocValuesField; +import org.apache.lucene.document.SortedNumericDocValuesField; +import org.apache.lucene.document.SortedSetDocValuesField; +import org.apache.lucene.document.StoredField; +import org.apache.lucene.document.StringField; +import org.apache.lucene.index.BaseCompressingDocValuesFormatTestCase; +import org.apache.lucene.index.BinaryDocValues; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.DocValues; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.IndexableField; +import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.NumericDocValues; +import org.apache.lucene.index.RandomIndexWriter; +import org.apache.lucene.index.SerialMergeScheduler; +import org.apache.lucene.index.SortedDocValues; +import org.apache.lucene.index.SortedNumericDocValues; +import org.apache.lucene.index.SortedSetDocValues; +import org.apache.lucene.index.Term; +import org.apache.lucene.index.Terms; +import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.index.TermsEnum.SeekStatus; +import org.apache.lucene.store.ByteBuffersDataInput; +import org.apache.lucene.store.ByteBuffersDataOutput; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefBuilder; +import org.apache.lucene.util.TestUtil; + +/** Tests Lucene90DocValuesFormat */ +public abstract class BaseLucene90DocValuesFormatTestCase + extends BaseCompressingDocValuesFormatTestCase { + + // TODO: these big methods can easily blow up some of the other ram-hungry codecs... + // for now just keep them here, as we want to test this for this format. + + public void testSortedSetVariableLengthBigVsStoredFields() throws Exception { + int numIterations = atLeast(1); + for (int i = 0; i < numIterations; i++) { + int numDocs = TEST_NIGHTLY ? atLeast(100) : atLeast(10); + doTestSortedSetVsStoredFields(numDocs, 1, 32766, 16, 100); + } + } + + @Nightly + public void testSortedSetVariableLengthManyVsStoredFields() throws Exception { + int numIterations = atLeast(1); + for (int i = 0; i < numIterations; i++) { + doTestSortedSetVsStoredFields(TestUtil.nextInt(random(), 1024, 2049), 1, 500, 16, 100); + } + } + + @Slow + public void testSortedVariableLengthBigVsStoredFields() throws Exception { + int numIterations = atLeast(1); + for (int i = 0; i < numIterations; i++) { + doTestSortedVsStoredFields(atLeast(100), 1d, 1, 32766); + } + } + + @Nightly + public void testSortedVariableLengthManyVsStoredFields() throws Exception { + int numIterations = atLeast(1); + for (int i = 0; i < numIterations; i++) { + doTestSortedVsStoredFields(TestUtil.nextInt(random(), 1024, 2049), 1d, 1, 500); + } + } + + @Nightly + public void testTermsEnumFixedWidth() throws Exception { + int numIterations = atLeast(1); + for (int i = 0; i < numIterations; i++) { + doTestTermsEnumRandom( + TestUtil.nextInt(random(), 1025, 5121), + () -> TestUtil.randomSimpleString(random(), 10, 10)); + } + } + + @Nightly + public void testTermsEnumVariableWidth() throws Exception { + int numIterations = atLeast(1); + for (int i = 0; i < numIterations; i++) { + doTestTermsEnumRandom( + TestUtil.nextInt(random(), 1025, 5121), + () -> TestUtil.randomSimpleString(random(), 1, 500)); + } + } + + @Nightly + public void testTermsEnumRandomMany() throws Exception { + int numIterations = atLeast(1); + for (int i = 0; i < numIterations; i++) { + doTestTermsEnumRandom( + TestUtil.nextInt(random(), 1025, 8121), + () -> TestUtil.randomSimpleString(random(), 1, 500)); + } + } + + @Nightly + public void testTermsEnumLongSharedPrefixes() throws Exception { + int numIterations = atLeast(1); + for (int i = 0; i < numIterations; i++) { + doTestTermsEnumRandom( + TestUtil.nextInt(random(), 1025, 5121), + () -> { + char[] chars = new char[random().nextInt(500)]; + Arrays.fill(chars, 'a'); + if (chars.length > 0) { + chars[random().nextInt(chars.length)] = 'b'; + } + return new String(chars); + }); + } + } + + @Slow + public void testSparseDocValuesVsStoredFields() throws Exception { + int numIterations = atLeast(1); + for (int i = 0; i < numIterations; i++) { + doTestSparseDocValuesVsStoredFields(); + } + } + + private void doTestSparseDocValuesVsStoredFields() throws Exception { + final long[] values = new long[TestUtil.nextInt(random(), 1, 500)]; + for (int i = 0; i < values.length; ++i) { + values[i] = random().nextLong(); + } + + Directory dir = newFSDirectory(createTempDir()); + IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random())); + conf.setMergeScheduler(new SerialMergeScheduler()); + RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf); + + // sparse compression is only enabled if less than 1% of docs have a value + final int avgGap = 100; + + final int numDocs = atLeast(200); + for (int i = random().nextInt(avgGap * 2); i >= 0; --i) { + writer.addDocument(new Document()); + } + final int maxNumValuesPerDoc = random().nextBoolean() ? 1 : TestUtil.nextInt(random(), 2, 5); + for (int i = 0; i < numDocs; ++i) { + Document doc = new Document(); + + // single-valued + long docValue = values[random().nextInt(values.length)]; + doc.add(new NumericDocValuesField("numeric", docValue)); + doc.add(new SortedDocValuesField("sorted", new BytesRef(Long.toString(docValue)))); + doc.add(new BinaryDocValuesField("binary", new BytesRef(Long.toString(docValue)))); + doc.add(new StoredField("value", docValue)); + + // multi-valued + final int numValues = TestUtil.nextInt(random(), 1, maxNumValuesPerDoc); + for (int j = 0; j < numValues; ++j) { + docValue = values[random().nextInt(values.length)]; + doc.add(new SortedNumericDocValuesField("sorted_numeric", docValue)); + doc.add(new SortedSetDocValuesField("sorted_set", new BytesRef(Long.toString(docValue)))); + doc.add(new StoredField("values", docValue)); + } + + writer.addDocument(doc); + + // add a gap + for (int j = TestUtil.nextInt(random(), 0, avgGap * 2); j >= 0; --j) { + writer.addDocument(new Document()); + } + } + + if (random().nextBoolean()) { + writer.forceMerge(1); + } + + final IndexReader indexReader = writer.getReader(); + writer.close(); + + for (LeafReaderContext context : indexReader.leaves()) { + final LeafReader reader = context.reader(); + final NumericDocValues numeric = DocValues.getNumeric(reader, "numeric"); + + final SortedDocValues sorted = DocValues.getSorted(reader, "sorted"); + + final BinaryDocValues binary = DocValues.getBinary(reader, "binary"); + + final SortedNumericDocValues sortedNumeric = + DocValues.getSortedNumeric(reader, "sorted_numeric"); + + final SortedSetDocValues sortedSet = DocValues.getSortedSet(reader, "sorted_set"); + + for (int i = 0; i < reader.maxDoc(); ++i) { + final Document doc = reader.document(i); + final IndexableField valueField = doc.getField("value"); + final Long value = valueField == null ? null : valueField.numericValue().longValue(); + + if (value == null) { + assertTrue(numeric.docID() + " vs " + i, numeric.docID() < i); + } else { + assertEquals(i, numeric.nextDoc()); + assertEquals(i, binary.nextDoc()); + assertEquals(i, sorted.nextDoc()); + assertEquals(value.longValue(), numeric.longValue()); + assertTrue(sorted.ordValue() >= 0); + assertEquals(new BytesRef(Long.toString(value)), sorted.lookupOrd(sorted.ordValue())); + assertEquals(new BytesRef(Long.toString(value)), binary.binaryValue()); + } + + final IndexableField[] valuesFields = doc.getFields("values"); + if (valuesFields.length == 0) { + assertTrue(sortedNumeric.docID() + " vs " + i, sortedNumeric.docID() < i); + } else { + final Set valueSet = new HashSet<>(); + for (IndexableField sf : valuesFields) { + valueSet.add(sf.numericValue().longValue()); + } + + assertEquals(i, sortedNumeric.nextDoc()); + assertEquals(valuesFields.length, sortedNumeric.docValueCount()); + for (int j = 0; j < sortedNumeric.docValueCount(); ++j) { + assertTrue(valueSet.contains(sortedNumeric.nextValue())); + } + assertEquals(i, sortedSet.nextDoc()); + int sortedSetCount = 0; + while (true) { + long ord = sortedSet.nextOrd(); + if (ord == SortedSetDocValues.NO_MORE_ORDS) { + break; + } + assertTrue(valueSet.contains(Long.parseLong(sortedSet.lookupOrd(ord).utf8ToString()))); + sortedSetCount++; + } + assertEquals(valueSet.size(), sortedSetCount); + } + } + } + + indexReader.close(); + dir.close(); + } + + // TODO: try to refactor this and some termsenum tests into the base class. + // to do this we need to fix the test class to get a DVF not a Codec so we can setup + // the postings format correctly. + private void doTestTermsEnumRandom(int numDocs, Supplier valuesProducer) + throws Exception { + Directory dir = newFSDirectory(createTempDir()); + IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random())); + conf.setMergeScheduler(new SerialMergeScheduler()); + // set to duel against a codec which has ordinals: + final PostingsFormat pf = TestUtil.getPostingsFormatWithOrds(random()); + final DocValuesFormat dv = + ((PerFieldDocValuesFormat) getCodec().docValuesFormat()) + .getDocValuesFormatForField("random_field_name"); + conf.setCodec( + new AssertingCodec() { + @Override + public PostingsFormat getPostingsFormatForField(String field) { + return pf; + } + + @Override + public DocValuesFormat getDocValuesFormatForField(String field) { + return dv; + } + }); + RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf); + + // index some docs + for (int i = 0; i < numDocs; i++) { + Document doc = new Document(); + Field idField = new StringField("id", Integer.toString(i), Field.Store.NO); + doc.add(idField); + int numValues = random().nextInt(17); + // create a random list of strings + List values = new ArrayList<>(); + for (int v = 0; v < numValues; v++) { + values.add(valuesProducer.get()); + } + + // add in any order to the indexed field + ArrayList unordered = new ArrayList<>(values); + Collections.shuffle(unordered, random()); + for (String v : values) { + doc.add(newStringField("indexed", v, Field.Store.NO)); + } + + // add in any order to the dv field + ArrayList unordered2 = new ArrayList<>(values); + Collections.shuffle(unordered2, random()); + for (String v : unordered2) { + doc.add(new SortedSetDocValuesField("dv", new BytesRef(v))); + } + + writer.addDocument(doc); + if (random().nextInt(31) == 0) { + writer.commit(); + } + } + + // delete some docs + int numDeletions = random().nextInt(numDocs / 10); + for (int i = 0; i < numDeletions; i++) { + int id = random().nextInt(numDocs); + writer.deleteDocuments(new Term("id", Integer.toString(id))); + } + + // compare per-segment + DirectoryReader ir = writer.getReader(); + for (LeafReaderContext context : ir.leaves()) { + LeafReader r = context.reader(); + Terms terms = r.terms("indexed"); + if (terms != null) { + SortedSetDocValues ssdv = r.getSortedSetDocValues("dv"); + assertEquals(terms.size(), ssdv.getValueCount()); + TermsEnum expected = terms.iterator(); + TermsEnum actual = r.getSortedSetDocValues("dv").termsEnum(); + assertEquals(terms.size(), expected, actual); + + doTestSortedSetEnumAdvanceIndependently(ssdv); + } + } + ir.close(); + + writer.forceMerge(1); + + // now compare again after the merge + ir = writer.getReader(); + LeafReader ar = getOnlyLeafReader(ir); + Terms terms = ar.terms("indexed"); + if (terms != null) { + assertEquals(terms.size(), ar.getSortedSetDocValues("dv").getValueCount()); + TermsEnum expected = terms.iterator(); + TermsEnum actual = ar.getSortedSetDocValues("dv").termsEnum(); + assertEquals(terms.size(), expected, actual); + } + ir.close(); + + writer.close(); + dir.close(); + } + + private void assertEquals(long numOrds, TermsEnum expected, TermsEnum actual) throws Exception { + BytesRef ref; + + // sequential next() through all terms + while ((ref = expected.next()) != null) { + assertEquals(ref, actual.next()); + assertEquals(expected.ord(), actual.ord()); + assertEquals(expected.term(), actual.term()); + } + assertNull(actual.next()); + + // sequential seekExact(ord) through all terms + for (long i = 0; i < numOrds; i++) { + expected.seekExact(i); + actual.seekExact(i); + assertEquals(expected.ord(), actual.ord()); + assertEquals(expected.term(), actual.term()); + } + + // sequential seekExact(BytesRef) through all terms + for (long i = 0; i < numOrds; i++) { + expected.seekExact(i); + assertTrue(actual.seekExact(expected.term())); + assertEquals(expected.ord(), actual.ord()); + assertEquals(expected.term(), actual.term()); + } + + // sequential seekCeil(BytesRef) through all terms + for (long i = 0; i < numOrds; i++) { + expected.seekExact(i); + assertEquals(SeekStatus.FOUND, actual.seekCeil(expected.term())); + assertEquals(expected.ord(), actual.ord()); + assertEquals(expected.term(), actual.term()); + } + + // random seekExact(ord) + for (long i = 0; i < numOrds; i++) { + long randomOrd = TestUtil.nextLong(random(), 0, numOrds - 1); + expected.seekExact(randomOrd); + actual.seekExact(randomOrd); + assertEquals(expected.ord(), actual.ord()); + assertEquals(expected.term(), actual.term()); + } + + // random seekExact(BytesRef) + for (long i = 0; i < numOrds; i++) { + long randomOrd = TestUtil.nextLong(random(), 0, numOrds - 1); + expected.seekExact(randomOrd); + actual.seekExact(expected.term()); + assertEquals(expected.ord(), actual.ord()); + assertEquals(expected.term(), actual.term()); + } + + // random seekCeil(BytesRef) + for (long i = 0; i < numOrds; i++) { + BytesRef target = new BytesRef(TestUtil.randomUnicodeString(random())); + SeekStatus expectedStatus = expected.seekCeil(target); + assertEquals(expectedStatus, actual.seekCeil(target)); + if (expectedStatus != SeekStatus.END) { + assertEquals(expected.ord(), actual.ord()); + assertEquals(expected.term(), actual.term()); + } + } + } + + @Nightly + public void testSortedSetAroundBlockSize() throws IOException { + final int frontier = 1 << Lucene90DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT; + for (int maxDoc = frontier - 1; maxDoc <= frontier + 1; ++maxDoc) { + final Directory dir = newDirectory(); + IndexWriter w = + new IndexWriter(dir, newIndexWriterConfig().setMergePolicy(newLogMergePolicy())); + ByteBuffersDataOutput out = new ByteBuffersDataOutput(); + Document doc = new Document(); + SortedSetDocValuesField field1 = new SortedSetDocValuesField("sset", new BytesRef()); + doc.add(field1); + SortedSetDocValuesField field2 = new SortedSetDocValuesField("sset", new BytesRef()); + doc.add(field2); + for (int i = 0; i < maxDoc; ++i) { + BytesRef s1 = new BytesRef(TestUtil.randomSimpleString(random(), 2)); + BytesRef s2 = new BytesRef(TestUtil.randomSimpleString(random(), 2)); + field1.setBytesValue(s1); + field2.setBytesValue(s2); + w.addDocument(doc); + Set set = new TreeSet<>(Arrays.asList(s1, s2)); + out.writeVInt(set.size()); + for (BytesRef ref : set) { + out.writeVInt(ref.length); + out.writeBytes(ref.bytes, ref.offset, ref.length); + } + } + + w.forceMerge(1); + DirectoryReader r = DirectoryReader.open(w); + w.close(); + LeafReader sr = getOnlyLeafReader(r); + assertEquals(maxDoc, sr.maxDoc()); + SortedSetDocValues values = sr.getSortedSetDocValues("sset"); + assertNotNull(values); + ByteBuffersDataInput in = out.toDataInput(); + BytesRefBuilder b = new BytesRefBuilder(); + for (int i = 0; i < maxDoc; ++i) { + assertEquals(i, values.nextDoc()); + final int numValues = in.readVInt(); + + for (int j = 0; j < numValues; ++j) { + b.setLength(in.readVInt()); + b.grow(b.length()); + in.readBytes(b.bytes(), 0, b.length()); + assertEquals(b.get(), values.lookupOrd(values.nextOrd())); + } + + assertEquals(SortedSetDocValues.NO_MORE_ORDS, values.nextOrd()); + } + r.close(); + dir.close(); + } + } + + @Nightly + public void testSortedNumericAroundBlockSize() throws IOException { + final int frontier = 1 << Lucene90DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT; + for (int maxDoc = frontier - 1; maxDoc <= frontier + 1; ++maxDoc) { + final Directory dir = newDirectory(); + IndexWriter w = + new IndexWriter(dir, newIndexWriterConfig().setMergePolicy(newLogMergePolicy())); + ByteBuffersDataOutput buffer = new ByteBuffersDataOutput(); + + Document doc = new Document(); + SortedNumericDocValuesField field1 = new SortedNumericDocValuesField("snum", 0L); + doc.add(field1); + SortedNumericDocValuesField field2 = new SortedNumericDocValuesField("snum", 0L); + doc.add(field2); + for (int i = 0; i < maxDoc; ++i) { + long s1 = random().nextInt(100); + long s2 = random().nextInt(100); + field1.setLongValue(s1); + field2.setLongValue(s2); + w.addDocument(doc); + buffer.writeVLong(Math.min(s1, s2)); + buffer.writeVLong(Math.max(s1, s2)); + } + + w.forceMerge(1); + DirectoryReader r = DirectoryReader.open(w); + w.close(); + LeafReader sr = getOnlyLeafReader(r); + assertEquals(maxDoc, sr.maxDoc()); + SortedNumericDocValues values = sr.getSortedNumericDocValues("snum"); + assertNotNull(values); + ByteBuffersDataInput dataInput = buffer.toDataInput(); + for (int i = 0; i < maxDoc; ++i) { + assertEquals(i, values.nextDoc()); + assertEquals(2, values.docValueCount()); + assertEquals(dataInput.readVLong(), values.nextValue()); + assertEquals(dataInput.readVLong(), values.nextValue()); + } + r.close(); + dir.close(); + } + } + + @Nightly + public void testSortedNumericBlocksOfVariousBitsPerValue() throws Exception { + doTestSortedNumericBlocksOfVariousBitsPerValue(() -> TestUtil.nextInt(random(), 1, 3)); + } + + @Nightly + public void testSparseSortedNumericBlocksOfVariousBitsPerValue() throws Exception { + doTestSortedNumericBlocksOfVariousBitsPerValue(() -> TestUtil.nextInt(random(), 0, 2)); + } + + @Nightly + public void testNumericBlocksOfVariousBitsPerValue() throws Exception { + doTestSparseNumericBlocksOfVariousBitsPerValue(1); + } + + @Nightly + public void testSparseNumericBlocksOfVariousBitsPerValue() throws Exception { + doTestSparseNumericBlocksOfVariousBitsPerValue(random().nextDouble()); + } + + // The LUCENE-8585 jump-tables enables O(1) skipping of IndexedDISI blocks, DENSE block lookup + // and numeric multi blocks. This test focuses on testing these jumps. + @Nightly + public void testNumericFieldJumpTables() throws Exception { + // IndexedDISI block skipping only activated if target >= current+2, so we need at least 5 + // blocks to + // trigger consecutive block skips + final int maxDoc = atLeast(5 * 65536); + + Directory dir = newDirectory(); + IndexWriter iw = createFastIndexWriter(dir, maxDoc); + + Field idField = newStringField("id", "", Field.Store.NO); + Field storedField = newStringField("stored", "", Field.Store.YES); + Field dvField = new NumericDocValuesField("dv", 0); + + for (int i = 0; i < maxDoc; i++) { + Document doc = new Document(); + idField.setStringValue(Integer.toBinaryString(i)); + doc.add(idField); + if (random().nextInt(100) > 10) { // Skip 10% to make DENSE blocks + int value = random().nextInt(100000); + storedField.setStringValue(Integer.toString(value)); + doc.add(storedField); + dvField.setLongValue(value); + doc.add(dvField); + } + iw.addDocument(doc); + } + iw.flush(); + iw.forceMerge(1, true); // Single segment to force large enough structures + iw.commit(); + iw.close(); + + assertDVIterate(dir); + assertDVAdvance( + dir, rarely() ? 1 : 7); // 1 is heavy (~20 s), so we do it rarely. 7 is a lot faster (8 s) + + dir.close(); + } + + private IndexWriter createFastIndexWriter(Directory dir, int maxBufferedDocs) throws IOException { + IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random())); + conf.setMaxBufferedDocs(maxBufferedDocs); + conf.setRAMBufferSizeMB(-1); + conf.setMergePolicy(newLogMergePolicy(random().nextBoolean())); + return new IndexWriter(dir, conf); + } + + private static LongSupplier blocksOfVariousBPV() { + final long mul = TestUtil.nextInt(random(), 1, 100); + final long min = random().nextInt(); + return new LongSupplier() { + int i = Lucene90DocValuesFormat.NUMERIC_BLOCK_SIZE; + int maxDelta; + + @Override + public long getAsLong() { + if (i == Lucene90DocValuesFormat.NUMERIC_BLOCK_SIZE) { + maxDelta = 1 << random().nextInt(5); + i = 0; + } + i++; + return min + mul * random().nextInt(maxDelta); + } + }; + } + + private void doTestSortedNumericBlocksOfVariousBitsPerValue(LongSupplier counts) + throws Exception { + Directory dir = newDirectory(); + IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random())); + conf.setMaxBufferedDocs(atLeast(Lucene90DocValuesFormat.NUMERIC_BLOCK_SIZE)); + conf.setRAMBufferSizeMB(-1); + conf.setMergePolicy(newLogMergePolicy(random().nextBoolean())); + IndexWriter writer = new IndexWriter(dir, conf); + + final int numDocs = atLeast(Lucene90DocValuesFormat.NUMERIC_BLOCK_SIZE * 3); + final LongSupplier values = blocksOfVariousBPV(); + for (int i = 0; i < numDocs; i++) { + Document doc = new Document(); + + int valueCount = (int) counts.getAsLong(); + long valueArray[] = new long[valueCount]; + for (int j = 0; j < valueCount; j++) { + long value = values.getAsLong(); + valueArray[j] = value; + doc.add(new SortedNumericDocValuesField("dv", value)); + } + Arrays.sort(valueArray); + for (int j = 0; j < valueCount; j++) { + doc.add(new StoredField("stored", Long.toString(valueArray[j]))); + } + writer.addDocument(doc); + if (random().nextInt(31) == 0) { + writer.commit(); + } + } + writer.forceMerge(1); + + writer.close(); + + // compare + DirectoryReader ir = DirectoryReader.open(dir); + TestUtil.checkReader(ir); + for (LeafReaderContext context : ir.leaves()) { + LeafReader r = context.reader(); + SortedNumericDocValues docValues = DocValues.getSortedNumeric(r, "dv"); + for (int i = 0; i < r.maxDoc(); i++) { + if (i > docValues.docID()) { + docValues.nextDoc(); + } + String expected[] = r.document(i).getValues("stored"); + if (i < docValues.docID()) { + assertEquals(0, expected.length); + } else { + String actual[] = new String[docValues.docValueCount()]; + for (int j = 0; j < actual.length; j++) { + actual[j] = Long.toString(docValues.nextValue()); + } + assertArrayEquals(expected, actual); + } + } + } + ir.close(); + dir.close(); + } + + private void doTestSparseNumericBlocksOfVariousBitsPerValue(double density) throws Exception { + Directory dir = newDirectory(); + IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random())); + conf.setMaxBufferedDocs(atLeast(Lucene90DocValuesFormat.NUMERIC_BLOCK_SIZE)); + conf.setRAMBufferSizeMB(-1); + conf.setMergePolicy(newLogMergePolicy(random().nextBoolean())); + IndexWriter writer = new IndexWriter(dir, conf); + Document doc = new Document(); + Field storedField = newStringField("stored", "", Field.Store.YES); + Field dvField = new NumericDocValuesField("dv", 0); + doc.add(storedField); + doc.add(dvField); + + final int numDocs = atLeast(Lucene90DocValuesFormat.NUMERIC_BLOCK_SIZE * 3); + final LongSupplier longs = blocksOfVariousBPV(); + for (int i = 0; i < numDocs; i++) { + if (random().nextDouble() > density) { + writer.addDocument(new Document()); + continue; + } + long value = longs.getAsLong(); + storedField.setStringValue(Long.toString(value)); + dvField.setLongValue(value); + writer.addDocument(doc); + } + + writer.forceMerge(1); + + writer.close(); + + // compare + assertDVIterate(dir); + assertDVAdvance( + dir, 1); // Tests all jump-lengths from 1 to maxDoc (quite slow ~= 1 minute for 200K docs) + + dir.close(); + } + + // Tests that advanceExact does not change the outcome + private void assertDVAdvance(Directory dir, int jumpStep) throws IOException { + DirectoryReader ir = DirectoryReader.open(dir); + TestUtil.checkReader(ir); + for (LeafReaderContext context : ir.leaves()) { + LeafReader r = context.reader(); + + for (int jump = jumpStep; jump < r.maxDoc(); jump += jumpStep) { + // Create a new instance each time to ensure jumps from the beginning + NumericDocValues docValues = DocValues.getNumeric(r, "dv"); + for (int docID = 0; docID < r.maxDoc(); docID += jump) { + String base = + "document #" + + docID + + "/" + + r.maxDoc() + + ", jumping " + + jump + + " from #" + + (docID - jump); + String storedValue = r.document(docID).get("stored"); + if (storedValue == null) { + assertFalse("There should be no DocValue for " + base, docValues.advanceExact(docID)); + } else { + assertTrue("There should be a DocValue for " + base, docValues.advanceExact(docID)); + assertEquals( + "The doc value should be correct for " + base, + Long.parseLong(storedValue), + docValues.longValue()); + } + } + } + } + ir.close(); + } +} diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestBestCompressionLucene90DocValuesFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestBestCompressionLucene90DocValuesFormat.java new file mode 100644 index 00000000000..8747701be60 --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestBestCompressionLucene90DocValuesFormat.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.lucene.codecs.lucene90; + +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.util.TestUtil; + +/** Tests Lucene90DocValuesFormat */ +public class TestBestCompressionLucene90DocValuesFormat + extends BaseLucene90DocValuesFormatTestCase { + private final Codec codec = + TestUtil.alwaysDocValuesFormat( + new Lucene90DocValuesFormat(Lucene90DocValuesFormat.Mode.BEST_COMPRESSION)); + + @Override + protected Codec getCodec() { + return codec; + } +} diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestBestSpeedLucene90DocValuesFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestBestSpeedLucene90DocValuesFormat.java new file mode 100644 index 00000000000..6303795f414 --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestBestSpeedLucene90DocValuesFormat.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.lucene.codecs.lucene90; + +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.util.TestUtil; + +/** Tests Lucene90DocValuesFormat */ +public class TestBestSpeedLucene90DocValuesFormat extends BaseLucene90DocValuesFormatTestCase { + private final Codec codec = + TestUtil.alwaysDocValuesFormat( + new Lucene90DocValuesFormat(Lucene90DocValuesFormat.Mode.BEST_SPEED)); + + @Override + protected Codec getCodec() { + return codec; + } +} diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestDocValuesCompression.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestDocValuesCompression.java new file mode 100644 index 00000000000..8df52604d4b --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestDocValuesCompression.java @@ -0,0 +1,313 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.lucene.codecs.lucene90; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import org.apache.lucene.analysis.Analyzer; +import org.apache.lucene.analysis.standard.StandardAnalyzer; +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.SortedDocValuesField; +import org.apache.lucene.document.SortedSetDocValuesField; +import org.apache.lucene.document.StringField; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.SortedDocValues; +import org.apache.lucene.index.SortedSetDocValues; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.LuceneTestCase; +import org.apache.lucene.util.TestUtil; + +public class TestDocValuesCompression extends LuceneTestCase { + private final Codec bestSpeed = new Lucene90Codec(Lucene90Codec.Mode.BEST_SPEED); + private final Codec bestCompression = new Lucene90Codec(Lucene90Codec.Mode.BEST_COMPRESSION); + + public void testTermsDictCompressionForLowCardinalityFields() throws IOException { + final int CARDINALITY = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_COMPRESSION_THRESHOLD - 1; + Set valuesSet = new HashSet<>(); + for (int i = 0; i < CARDINALITY; ++i) { + final int length = TestUtil.nextInt(random(), 10, 30); + String value = TestUtil.randomSimpleString(random(), length); + valuesSet.add(value); + } + + List values = new ArrayList<>(valuesSet); + long sizeForBestSpeed = writeAndGetDocValueFileSize(bestSpeed, values); + long sizeForBestCompression = writeAndGetDocValueFileSize(bestCompression, values); + + // Ensure terms dict data was not compressed for low-cardinality fields. + assertEquals(sizeForBestSpeed, sizeForBestCompression); + } + + public void testTermsDictCompressionForHighCardinalityFields() throws IOException { + final int CARDINALITY = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_COMPRESSION_THRESHOLD << 1; + Set valuesSet = new HashSet<>(); + for (int i = 0; i < CARDINALITY; ++i) { + final int length = TestUtil.nextInt(random(), 10, 30); + String value = TestUtil.randomSimpleString(random(), length); + // Add common suffix for better compression ratio. + valuesSet.add(value + "_CommonPartBetterForCompression"); + } + + List values = new ArrayList<>(valuesSet); + long sizeForBestSpeed = writeAndGetDocValueFileSize(bestSpeed, values); + long sizeForBestCompression = writeAndGetDocValueFileSize(bestCompression, values); + + // Compression happened. + assertTrue(sizeForBestCompression < sizeForBestSpeed); + } + + public void testReseekAfterSkipDecompression() throws IOException { + final int CARDINALITY = (Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_SIZE << 1) + 11; + Set valueSet = new HashSet<>(CARDINALITY); + for (int i = 0; i < CARDINALITY; i++) { + valueSet.add(TestUtil.randomSimpleString(random(), 64)); + } + List values = new ArrayList<>(valueSet); + Collections.sort(values); + // Create one non-existent value just between block-1 and block-2. + String nonexistentValue = + values.get(Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_SIZE - 1) + + TestUtil.randomSimpleString(random(), 64, 128); + int docValues = values.size(); + + try (Directory directory = newDirectory()) { + Analyzer analyzer = new StandardAnalyzer(); + IndexWriterConfig config = new IndexWriterConfig(analyzer); + config.setCodec(bestCompression); + config.setUseCompoundFile(false); + IndexWriter writer = new IndexWriter(directory, config); + for (int i = 0; i < 280; i++) { + Document doc = new Document(); + doc.add(new StringField("id", "Doc" + i, Field.Store.NO)); + doc.add(new SortedDocValuesField("sdv", new BytesRef(values.get(i % docValues)))); + writer.addDocument(doc); + } + writer.commit(); + writer.forceMerge(1); + DirectoryReader dReader = DirectoryReader.open(writer); + writer.close(); + + LeafReader reader = getOnlyLeafReader(dReader); + // Check values count. + SortedDocValues ssdvMulti = reader.getSortedDocValues("sdv"); + assertEquals(docValues, ssdvMulti.getValueCount()); + + // Seek to first block. + int ord1 = ssdvMulti.lookupTerm(new BytesRef(values.get(0))); + assertTrue(ord1 >= 0); + int ord2 = ssdvMulti.lookupTerm(new BytesRef(values.get(1))); + assertTrue(ord2 >= ord1); + // Ensure re-seek logic is correct after skip-decompression. + int nonexistentOrd2 = ssdvMulti.lookupTerm(new BytesRef(nonexistentValue)); + assertTrue(nonexistentOrd2 < 0); + dReader.close(); + } + } + + public void testLargeTermsCompression() throws IOException { + final int CARDINALITY = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_COMPRESSION_THRESHOLD << 1; + Set valuesSet = new HashSet<>(); + for (int i = 0; i < CARDINALITY; ++i) { + final int length = TestUtil.nextInt(random(), 512, 1024); + valuesSet.add(TestUtil.randomSimpleString(random(), length)); + } + int valuesCount = valuesSet.size(); + List values = new ArrayList<>(valuesSet); + + try (Directory directory = newDirectory()) { + Analyzer analyzer = new StandardAnalyzer(); + IndexWriterConfig config = new IndexWriterConfig(analyzer); + config.setCodec(bestCompression); + config.setUseCompoundFile(false); + IndexWriter writer = new IndexWriter(directory, config); + for (int i = 0; i < 256; i++) { + Document doc = new Document(); + doc.add(new StringField("id", "Doc" + i, Field.Store.NO)); + doc.add(new SortedDocValuesField("sdv", new BytesRef(values.get(i % valuesCount)))); + writer.addDocument(doc); + } + writer.commit(); + writer.forceMerge(1); + DirectoryReader ireader = DirectoryReader.open(writer); + writer.close(); + + LeafReader reader = getOnlyLeafReader(ireader); + // Check values count. + SortedDocValues ssdvMulti = reader.getSortedDocValues("sdv"); + assertEquals(valuesCount, ssdvMulti.getValueCount()); + ireader.close(); + } + } + + // Ensure the old segment can be merged together with the new compressed segment. + public void testMergeWithUncompressedSegment() throws IOException { + final int CARDINALITY = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_COMPRESSION_THRESHOLD << 1; + Set valuesSet = new HashSet<>(); + for (int i = 0; i < CARDINALITY; ++i) { + final int length = TestUtil.nextInt(random(), 10, 30); + // Add common suffix for better compression ratio. + valuesSet.add(TestUtil.randomSimpleString(random(), length)); + } + List values = new ArrayList<>(valuesSet); + int valuesCount = values.size(); + + try (Directory directory = newDirectory()) { + // 1. Write 256 documents without terms dict compression. + Analyzer analyzer = new StandardAnalyzer(); + IndexWriterConfig config = new IndexWriterConfig(analyzer); + config.setCodec(bestSpeed); + config.setUseCompoundFile(false); + IndexWriter writer = new IndexWriter(directory, config); + for (int i = 0; i < 256; i++) { + Document doc = new Document(); + doc.add(new StringField("id", "Doc" + i, Field.Store.NO)); + doc.add(new SortedSetDocValuesField("ssdv", new BytesRef(values.get(i % valuesCount)))); + doc.add( + new SortedSetDocValuesField("ssdv", new BytesRef(values.get((i + 1) % valuesCount)))); + doc.add( + new SortedSetDocValuesField("ssdv", new BytesRef(values.get((i + 2) % valuesCount)))); + doc.add(new SortedDocValuesField("sdv", new BytesRef(values.get(i % valuesCount)))); + writer.addDocument(doc); + } + writer.commit(); + DirectoryReader ireader = DirectoryReader.open(writer); + assertEquals(256, ireader.numDocs()); + LeafReader reader = getOnlyLeafReader(ireader); + SortedSetDocValues ssdv = reader.getSortedSetDocValues("ssdv"); + assertEquals(valuesCount, ssdv.getValueCount()); + SortedDocValues sdv = reader.getSortedDocValues("sdv"); + assertEquals(valuesCount, sdv.getValueCount()); + ireader.close(); + writer.close(); + + // 2. Add another 100 documents, and enabling terms dict compression. + config = new IndexWriterConfig(analyzer); + config.setCodec(bestCompression); + config.setUseCompoundFile(false); + writer = new IndexWriter(directory, config); + // Add 2 new values. + valuesSet.add(TestUtil.randomSimpleString(random(), 10)); + valuesSet.add(TestUtil.randomSimpleString(random(), 10)); + values = new ArrayList<>(valuesSet); + valuesCount = valuesSet.size(); + + for (int i = 256; i < 356; i++) { + Document doc = new Document(); + doc.add(new StringField("id", "Doc" + i, Field.Store.NO)); + doc.add(new SortedSetDocValuesField("ssdv", new BytesRef(values.get(i % valuesCount)))); + doc.add(new SortedDocValuesField("sdv", new BytesRef(values.get(i % valuesCount)))); + writer.addDocument(doc); + } + writer.commit(); + writer.forceMerge(1); + ireader = DirectoryReader.open(writer); + assertEquals(356, ireader.numDocs()); + reader = getOnlyLeafReader(ireader); + ssdv = reader.getSortedSetDocValues("ssdv"); + assertEquals(valuesCount, ssdv.getValueCount()); + ireader.close(); + writer.close(); + } + } + + private static long writeAndGetDocValueFileSize(Codec codec, List values) + throws IOException { + int valuesCount = values.size(); + long dvdFileSize = -1; + try (Directory directory = newDirectory()) { + Analyzer analyzer = new StandardAnalyzer(); + IndexWriterConfig config = new IndexWriterConfig(analyzer); + config.setCodec(codec); + config.setUseCompoundFile(false); + IndexWriter writer = new IndexWriter(directory, config); + for (int i = 0; i < 256; i++) { + Document doc = new Document(); + doc.add(new StringField("id", "Doc" + i, Field.Store.NO)); + // Multi value sorted-set field. + doc.add( + new SortedSetDocValuesField("ssdv_multi_", new BytesRef(values.get(i % valuesCount)))); + doc.add( + new SortedSetDocValuesField( + "ssdv_multi_", new BytesRef(values.get((i + 1) % valuesCount)))); + doc.add( + new SortedSetDocValuesField( + "ssdv_multi_", new BytesRef(values.get((i + 2) % valuesCount)))); + // Single value sorted-set field. + doc.add( + new SortedSetDocValuesField("ssdv_single_", new BytesRef(values.get(i % valuesCount)))); + // Sorted field. + doc.add(new SortedDocValuesField("sdv", new BytesRef(values.get(i % valuesCount)))); + writer.addDocument(doc); + } + writer.commit(); + writer.forceMerge(1); + DirectoryReader ireader = DirectoryReader.open(writer); + writer.close(); + + LeafReader reader = getOnlyLeafReader(ireader); + // Check values count. + SortedSetDocValues ssdvMulti = reader.getSortedSetDocValues("ssdv_multi_"); + assertEquals(valuesCount, ssdvMulti.getValueCount()); + for (int i = 0; i < valuesCount; i++) { + BytesRef term = ssdvMulti.lookupOrd(i); + assertTrue(term.bytes.length > 0); + } + for (int i = 0; i < valuesCount; i++) { + for (int j = 0; j < 3; j++) { + assertTrue(ssdvMulti.lookupTerm(new BytesRef(values.get((i + j) % valuesCount))) >= 0); + } + } + + SortedSetDocValues ssdvSingle = reader.getSortedSetDocValues("ssdv_single_"); + assertEquals(valuesCount, ssdvSingle.getValueCount()); + for (int i = 0; i < valuesCount; i++) { + assertTrue(ssdvSingle.lookupTerm(new BytesRef(values.get(i % valuesCount))) >= 0); + } + + SortedDocValues sdv = reader.getSortedDocValues("sdv"); + assertEquals(valuesCount, sdv.getValueCount()); + for (int i = 0; i < valuesCount; i++) { + assertTrue(sdv.lookupTerm(new BytesRef(values.get(i % valuesCount))) >= 0); + } + + dvdFileSize = docValueFileSize(directory); + assertTrue(dvdFileSize > 0); + ireader.close(); + } + + return dvdFileSize; + } + + static long docValueFileSize(Directory d) throws IOException { + for (String file : d.listAll()) { + if (file.endsWith(Lucene90DocValuesFormat.DATA_EXTENSION)) { + return d.fileLength(file); + } + } + return -1; + } +} diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestIndexedDISI.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestIndexedDISI.java similarity index 99% rename from lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestIndexedDISI.java rename to lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestIndexedDISI.java index cd613dbb715..97f22bdff38 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestIndexedDISI.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestIndexedDISI.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene80; +package org.apache.lucene.codecs.lucene90; import java.io.IOException; import java.util.Random; diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java b/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java index 171384a56c7..2ecfe45fcef 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java +++ b/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java @@ -39,9 +39,9 @@ import org.apache.lucene.codecs.blockterms.LuceneVarGapDocFreqInterval; import org.apache.lucene.codecs.blockterms.LuceneVarGapFixedInterval; import org.apache.lucene.codecs.blocktreeords.BlockTreeOrdsPostingsFormat; import org.apache.lucene.codecs.bloom.TestBloomFilteredLucenePostings; -import org.apache.lucene.codecs.lucene80.Lucene80DocValuesFormat; import org.apache.lucene.codecs.lucene86.Lucene86PointsReader; import org.apache.lucene.codecs.lucene86.Lucene86PointsWriter; +import org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat; import org.apache.lucene.codecs.memory.DirectPostingsFormat; import org.apache.lucene.codecs.memory.FSTPostingsFormat; import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat; @@ -227,7 +227,7 @@ public class RandomCodec extends AssertingCodec { addDocValues( avoidCodecs, TestUtil.getDefaultDocValuesFormat(), - new Lucene80DocValuesFormat(Lucene80DocValuesFormat.Mode.BEST_COMPRESSION), + new Lucene90DocValuesFormat(Lucene90DocValuesFormat.Mode.BEST_COMPRESSION), new AssertingDocValuesFormat()); Collections.shuffle(formats, random); 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 d6615d5c9a3..8b697439a10 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 @@ -53,8 +53,8 @@ import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.asserting.AssertingCodec; 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.lucene90.Lucene90Codec; +import org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat; import org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat; import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat; import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; @@ -1252,7 +1252,7 @@ public final class TestUtil { * Lucene. */ public static DocValuesFormat getDefaultDocValuesFormat() { - return new Lucene80DocValuesFormat(); + return new Lucene90DocValuesFormat(); } // TODO: generalize all 'test-checks-for-crazy-codecs' to