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:
+ *
+ *
+ * - {@code ALL} if the range contains 65536 documents exactly,
+ *
- {@code DENSE} if the range contains 4096 documents or more; in that case documents are
+ * stored in a bit set,
+ *
- {@code SPARSE} otherwise, and the lower 16 bits of the doc IDs are stored in a {@link
+ * DataInput#readShort() short}.
+ *
+ *
+ * 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:
+ *
+ *
+ * - SPARSE: This strategy is used when a block contains at most 4095 documents. The lower 16
+ * bits of doc IDs are stored as {@link DataOutput#writeShort(short) shorts} while the upper
+ * 16 bits are given by the block ID.
+ *
- DENSE: This strategy is used when a block contains between 4096 and 65535 documents. The
+ * lower bits of doc IDs are stored in a bit set. Advancing < 512 documents is performed
+ * using {@link Long#numberOfTrailingZeros(long) ntz} operations while the index is computed
+ * by accumulating the {@link Long#bitCount(long) bit counts} of the visited longs. Advancing
+ * >= 512 documents is performed by skipping to the start of the needed 512 document
+ * sub-block and iterating to the specific document within that block. The index for the
+ * sub-block that is skipped to is retrieved from a rank-table positioned beforethe bit set.
+ * The rank-table holds the origo index numbers for all 512 documents sub-blocks, represented
+ * as an unsigned short for each 128 blocks.
+ *
- ALL: This strategy is used when a block contains exactly 65536 documents, meaning that the
+ * block is full. In that case doc IDs do not need to be stored explicitly. This is typically
+ * faster than both SPARSE and DENSE which is a reason why it is preferable to have all
+ * documents that have a value for a field using contiguous doc IDs, for instance by using
+ * {@link IndexWriterConfig#setIndexSort(org.apache.lucene.search.Sort) index sorting}.
+ *
+ *
+ * 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}:
+ *
+ *
+ * - Delta-compressed: per-document integers written as deltas from the minimum value,
+ * compressed with bitpacking. For more information, see {@link DirectWriter}.
+ *
- Table-compressed: when the number of unique values is very small (< 256), and when there
+ * are unused "gaps" in the range of values used (such as {@link SmallFloat}), a lookup table
+ * is written instead. Each per-document entry is instead the ordinal to this table, and those
+ * ordinals are compressed with bitpacking ({@link DirectWriter}).
+ *
- GCD-compressed: when all numbers share a common divisor, such as dates, the greatest common
+ * denominator (GCD) is computed, and quotients are stored using Delta-compressed Numerics.
+ *
- Monotonic-compressed: when all numbers are monotonically increasing offsets, they are
+ * written as blocks of bitpacked integers, encoding the deviation from the expected delta.
+ *
- Const-compressed: when there is only one possible value, no per-document data is needed and
+ * this value is encoded alone.
+ *
+ *
+ * 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}:
+ *
+ *
+ * - Fixed-width Binary: one large concatenated byte[] is written, along with the fixed length.
+ * Each document's value can be addressed directly with multiplication ({@code docID *
+ * length}).
+ *
- Variable-width Binary: one large concatenated byte[] is written, along with end addresses
+ * for each document. The addresses are written as Monotonic-compressed numerics.
+ *
- Prefix-compressed Binary: values are written in chunks of 16, with the first value written
+ * completely and other values sharing prefixes. chunk addresses are written as
+ * Monotonic-compressed numerics. A reverse lookup index is written from a portion of every
+ * 1024th term.
+ *
+ *
+ * {@link DocValuesType#SORTED SORTED}:
+ *
+ *
+ * - Sorted: a mapping of ordinals to deduplicated terms is written as Prefix-compressed Binary,
+ * along with the per-document ordinals written using one of the numeric strategies above.
+ *
+ *
+ * {@link DocValuesType#SORTED_SET SORTED_SET}:
+ *
+ *
+ * - Single: if all documents have 0 or 1 value, then data are written like SORTED.
+ *
- SortedSet: a mapping of ordinals to deduplicated terms is written as Binary, an ordinal
+ * list and per-document index into this list are written using the numeric strategies above.
+ *
+ *
+ * {@link DocValuesType#SORTED_NUMERIC SORTED_NUMERIC}:
+ *
+ *
+ * - Single: if all documents have 0 or 1 value, then data are written like NUMERIC.
+ *
- SortedNumeric: a value list and per-document index into this list are written using the
+ * numeric strategies above.
+ *
+ *
+ * Files:
+ *
+ *
+ * .dvd
: DocValues data
+ * .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:
+ *
+ *
+ * .nvd
: Norms data
+ * .nvm
: Norms metadata
+ *
+ *
+ *
+ * -
+ *
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.
+ *
-
+ *
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