mirror of https://github.com/apache/lucene.git
LUCENE-9705: Create Lucene90DocValuesFormat and Lucene90NormsFormat (#2392)
For now these are just copies of Lucene80DocValuesFormat and Lucene80NormsFormat. The existing formats were moved to backwards-codecs.
This commit is contained in:
parent
42da2b45e6
commit
4d7b2aebfe
|
@ -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(
|
||||
|
|
|
@ -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.
|
||||
*
|
||||
* <p>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:
|
||||
*
|
||||
* <ul>
|
||||
* <li>{@code ALL} if the range contains 65536 documents exactly,
|
||||
* <li>{@code DENSE} if the range contains 4096 documents or more; in that case documents are
|
||||
* stored in a bit set,
|
||||
* <li>{@code SPARSE} otherwise, and the lower 16 bits of the doc IDs are stored in a {@link
|
||||
* DataInput#readShort() short}.
|
||||
* </ul>
|
||||
*
|
||||
* <p>Only ranges that contain at least one value are encoded.
|
||||
*
|
||||
* <p>This implementation uses 6 bytes per document in the worst-case, which happens in the case
|
||||
* that all ranges contain exactly one document.
|
||||
*
|
||||
* <p>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.
|
||||
*
|
||||
* <p>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.
|
||||
*
|
||||
* <p>Each int-pair entry consists of 2 logical parts:
|
||||
*
|
||||
* <p>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.
|
||||
*
|
||||
* <p>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.
|
||||
*
|
||||
* <p>The cache overhead is numDocs/1024 bytes.
|
||||
*
|
||||
* <p>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.
|
||||
*
|
||||
* <p>The block lookup table is stored at the end of the total block structure.
|
||||
*
|
||||
* <p>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.
|
||||
*
|
||||
* <p>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.
|
||||
*
|
||||
* <p>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;
|
||||
}
|
||||
}
|
|
@ -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;
|
|||
*
|
||||
* <p>
|
||||
*
|
||||
* @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 {
|
||||
|
|
|
@ -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();
|
|
@ -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(
|
|
@ -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;
|
|
@ -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;
|
||||
}
|
|
@ -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;
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
|||
*
|
||||
* <p>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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -14,3 +14,4 @@
|
|||
# limitations under the License.
|
||||
|
||||
org.apache.lucene.backward_codecs.lucene70.Lucene70DocValuesFormat
|
||||
org.apache.lucene.backward_codecs.lucene80.Lucene80DocValuesFormat
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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;
|
|
@ -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;
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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;
|
|
@ -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;
|
|
@ -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;
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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() {
|
|
@ -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 {
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
|
@ -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() {
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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.
|
||||
*
|
||||
* <p>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 <code>{1, 5, 6, 11}</code>. When the
|
||||
* iterator is on <code>6</code>, 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:
|
||||
*
|
||||
* <ul>
|
||||
* <li>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.
|
||||
* <li>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.
|
||||
* <li>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}.
|
||||
* </ul>
|
||||
*
|
||||
* <p>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.
|
||||
*
|
||||
* <p>Then the five per-document value types (Numeric,Binary,Sorted,SortedSet,SortedNumeric) are
|
||||
* encoded using the following strategies:
|
||||
*
|
||||
* <p>{@link DocValuesType#NUMERIC NUMERIC}:
|
||||
*
|
||||
* <ul>
|
||||
* <li>Delta-compressed: per-document integers written as deltas from the minimum value,
|
||||
* compressed with bitpacking. For more information, see {@link DirectWriter}.
|
||||
* <li>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}).
|
||||
* <li>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.
|
||||
* <li>Monotonic-compressed: when all numbers are monotonically increasing offsets, they are
|
||||
* written as blocks of bitpacked integers, encoding the deviation from the expected delta.
|
||||
* <li>Const-compressed: when there is only one possible value, no per-document data is needed and
|
||||
* this value is encoded alone.
|
||||
* </ul>
|
||||
*
|
||||
* <p>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.
|
||||
*
|
||||
* <p>{@link DocValuesType#BINARY BINARY}:
|
||||
*
|
||||
* <ul>
|
||||
* <li>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}).
|
||||
* <li>Variable-width Binary: one large concatenated byte[] is written, along with end addresses
|
||||
* for each document. The addresses are written as Monotonic-compressed numerics.
|
||||
* <li>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.
|
||||
* </ul>
|
||||
*
|
||||
* <p>{@link DocValuesType#SORTED SORTED}:
|
||||
*
|
||||
* <ul>
|
||||
* <li>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.
|
||||
* </ul>
|
||||
*
|
||||
* <p>{@link DocValuesType#SORTED_SET SORTED_SET}:
|
||||
*
|
||||
* <ul>
|
||||
* <li>Single: if all documents have 0 or 1 value, then data are written like SORTED.
|
||||
* <li>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.
|
||||
* </ul>
|
||||
*
|
||||
* <p>{@link DocValuesType#SORTED_NUMERIC SORTED_NUMERIC}:
|
||||
*
|
||||
* <ul>
|
||||
* <li>Single: if all documents have 0 or 1 value, then data are written like NUMERIC.
|
||||
* <li>SortedNumeric: a value list and per-document index into this list are written using the
|
||||
* numeric strategies above.
|
||||
* </ul>
|
||||
*
|
||||
* <p>Files:
|
||||
*
|
||||
* <ol>
|
||||
* <li><code>.dvd</code>: DocValues data
|
||||
* <li><code>.dvm</code>: DocValues metadata
|
||||
* </ol>
|
||||
*
|
||||
* @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;
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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.
|
||||
*
|
||||
* <p>Encodes normalization values by encoding each value with the minimum number of bytes needed to
|
||||
* represent the range (which can be zero).
|
||||
*
|
||||
* <p>Files:
|
||||
*
|
||||
* <ol>
|
||||
* <li><code>.nvd</code>: Norms data
|
||||
* <li><code>.nvm</code>: Norms metadata
|
||||
* </ol>
|
||||
*
|
||||
* <ol>
|
||||
* <li><a id="nvm"></a>
|
||||
* <p>The Norms metadata or .nvm file.
|
||||
* <p>For each norms field, this stores metadata, such as the offset into the Norms data
|
||||
* (.nvd)
|
||||
* <p>Norms metadata (.dvm) --> Header,<Entry><sup>NumFields</sup>,Footer
|
||||
* <ul>
|
||||
* <li>Header --> {@link CodecUtil#writeIndexHeader IndexHeader}
|
||||
* <li>Entry --> FieldNumber, DocsWithFieldAddress, DocsWithFieldLength,
|
||||
* NumDocsWithField, BytesPerNorm, NormsAddress
|
||||
* <li>FieldNumber --> {@link DataOutput#writeInt Int32}
|
||||
* <li>DocsWithFieldAddress --> {@link DataOutput#writeLong Int64}
|
||||
* <li>DocsWithFieldLength --> {@link DataOutput#writeLong Int64}
|
||||
* <li>NumDocsWithField --> {@link DataOutput#writeInt Int32}
|
||||
* <li>BytesPerNorm --> {@link DataOutput#writeByte byte}
|
||||
* <li>NormsAddress --> {@link DataOutput#writeLong Int64}
|
||||
* <li>Footer --> {@link CodecUtil#writeFooter CodecFooter}
|
||||
* </ul>
|
||||
* <p>FieldNumber of -1 indicates the end of metadata.
|
||||
* <p>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.
|
||||
* <p>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.
|
||||
* <p>DocsWithFieldLength is the number of bytes used to encode the set of documents that have
|
||||
* a norm.
|
||||
* <li><a id="nvd"></a>
|
||||
* <p>The Norms data or .nvd file.
|
||||
* <p>For each Norms field, this stores the actual per-document data (the heavy-lifting)
|
||||
* <p>Norms data (.nvd) --> Header,< Data ><sup>NumFields</sup>,Footer
|
||||
* <ul>
|
||||
* <li>Header --> {@link CodecUtil#writeIndexHeader IndexHeader}
|
||||
* <li>DocsWithFieldData --> {@link IndexedDISI#writeBitSet Bit set of MaxDoc bits}
|
||||
* <li>NormsData --> {@link DataOutput#writeByte(byte) byte}<sup>NumDocsWithField *
|
||||
* BytesPerValue</sup>
|
||||
* <li>Footer --> {@link CodecUtil#writeFooter CodecFooter}
|
||||
* </ul>
|
||||
* </ol>
|
||||
*
|
||||
* @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;
|
||||
}
|
|
@ -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<Integer, NormsEntry> norms = new HashMap<>();
|
||||
private final int maxDoc;
|
||||
private IndexInput data;
|
||||
private boolean merging;
|
||||
private Map<Integer, IndexInput> disiInputs;
|
||||
private Map<Integer, RandomAccessInput> disiJumpTables;
|
||||
private Map<Integer, RandomAccessInput> 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() + ")";
|
||||
}
|
||||
}
|
|
@ -162,14 +162,14 @@
|
|||
* <li>{@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.
|
||||
* <li>{@link org.apache.lucene.codecs.lucene80.Lucene80NormsFormat Normalization factors}. For
|
||||
* <li>{@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.
|
||||
* <li>{@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
|
||||
* <li>{@link org.apache.lucene.codecs.lucene80.Lucene80DocValuesFormat Per-document values}. Like
|
||||
* <li>{@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 @@
|
|||
* <td>Stores additional per-position metadata information such as character offsets and user payloads</td>
|
||||
* </tr>
|
||||
* <tr>
|
||||
* <td>{@link org.apache.lucene.codecs.lucene80.Lucene80NormsFormat Norms}</td>
|
||||
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90NormsFormat Norms}</td>
|
||||
* <td>.nvd, .nvm</td>
|
||||
* <td>Encodes length and boost factors for docs and fields</td>
|
||||
* </tr>
|
||||
* <tr>
|
||||
* <td>{@link org.apache.lucene.codecs.lucene80.Lucene80DocValuesFormat Per-Document Values}</td>
|
||||
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat Per-Document Values}</td>
|
||||
* <td>.dvd, .dvm</td>
|
||||
* <td>Encodes additional scoring factors or other per-document information.</td>
|
||||
* </tr>
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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<Long> 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<String> 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<String> values = new ArrayList<>();
|
||||
for (int v = 0; v < numValues; v++) {
|
||||
values.add(valuesProducer.get());
|
||||
}
|
||||
|
||||
// add in any order to the indexed field
|
||||
ArrayList<String> 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<String> 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<BytesRef> 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();
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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<String> valuesSet = new HashSet<>();
|
||||
for (int i = 0; i < CARDINALITY; ++i) {
|
||||
final int length = TestUtil.nextInt(random(), 10, 30);
|
||||
String value = TestUtil.randomSimpleString(random(), length);
|
||||
valuesSet.add(value);
|
||||
}
|
||||
|
||||
List<String> values = new ArrayList<>(valuesSet);
|
||||
long sizeForBestSpeed = writeAndGetDocValueFileSize(bestSpeed, values);
|
||||
long sizeForBestCompression = writeAndGetDocValueFileSize(bestCompression, values);
|
||||
|
||||
// Ensure terms dict data was not compressed for low-cardinality fields.
|
||||
assertEquals(sizeForBestSpeed, sizeForBestCompression);
|
||||
}
|
||||
|
||||
public void testTermsDictCompressionForHighCardinalityFields() throws IOException {
|
||||
final int CARDINALITY = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_COMPRESSION_THRESHOLD << 1;
|
||||
Set<String> valuesSet = new HashSet<>();
|
||||
for (int i = 0; i < CARDINALITY; ++i) {
|
||||
final int length = TestUtil.nextInt(random(), 10, 30);
|
||||
String value = TestUtil.randomSimpleString(random(), length);
|
||||
// Add common suffix for better compression ratio.
|
||||
valuesSet.add(value + "_CommonPartBetterForCompression");
|
||||
}
|
||||
|
||||
List<String> values = new ArrayList<>(valuesSet);
|
||||
long sizeForBestSpeed = writeAndGetDocValueFileSize(bestSpeed, values);
|
||||
long sizeForBestCompression = writeAndGetDocValueFileSize(bestCompression, values);
|
||||
|
||||
// Compression happened.
|
||||
assertTrue(sizeForBestCompression < sizeForBestSpeed);
|
||||
}
|
||||
|
||||
public void testReseekAfterSkipDecompression() throws IOException {
|
||||
final int CARDINALITY = (Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_SIZE << 1) + 11;
|
||||
Set<String> valueSet = new HashSet<>(CARDINALITY);
|
||||
for (int i = 0; i < CARDINALITY; i++) {
|
||||
valueSet.add(TestUtil.randomSimpleString(random(), 64));
|
||||
}
|
||||
List<String> values = new ArrayList<>(valueSet);
|
||||
Collections.sort(values);
|
||||
// Create one non-existent value just between block-1 and block-2.
|
||||
String nonexistentValue =
|
||||
values.get(Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_SIZE - 1)
|
||||
+ TestUtil.randomSimpleString(random(), 64, 128);
|
||||
int docValues = values.size();
|
||||
|
||||
try (Directory directory = newDirectory()) {
|
||||
Analyzer analyzer = new StandardAnalyzer();
|
||||
IndexWriterConfig config = new IndexWriterConfig(analyzer);
|
||||
config.setCodec(bestCompression);
|
||||
config.setUseCompoundFile(false);
|
||||
IndexWriter writer = new IndexWriter(directory, config);
|
||||
for (int i = 0; i < 280; i++) {
|
||||
Document doc = new Document();
|
||||
doc.add(new StringField("id", "Doc" + i, Field.Store.NO));
|
||||
doc.add(new SortedDocValuesField("sdv", new BytesRef(values.get(i % docValues))));
|
||||
writer.addDocument(doc);
|
||||
}
|
||||
writer.commit();
|
||||
writer.forceMerge(1);
|
||||
DirectoryReader dReader = DirectoryReader.open(writer);
|
||||
writer.close();
|
||||
|
||||
LeafReader reader = getOnlyLeafReader(dReader);
|
||||
// Check values count.
|
||||
SortedDocValues ssdvMulti = reader.getSortedDocValues("sdv");
|
||||
assertEquals(docValues, ssdvMulti.getValueCount());
|
||||
|
||||
// Seek to first block.
|
||||
int ord1 = ssdvMulti.lookupTerm(new BytesRef(values.get(0)));
|
||||
assertTrue(ord1 >= 0);
|
||||
int ord2 = ssdvMulti.lookupTerm(new BytesRef(values.get(1)));
|
||||
assertTrue(ord2 >= ord1);
|
||||
// Ensure re-seek logic is correct after skip-decompression.
|
||||
int nonexistentOrd2 = ssdvMulti.lookupTerm(new BytesRef(nonexistentValue));
|
||||
assertTrue(nonexistentOrd2 < 0);
|
||||
dReader.close();
|
||||
}
|
||||
}
|
||||
|
||||
public void testLargeTermsCompression() throws IOException {
|
||||
final int CARDINALITY = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_COMPRESSION_THRESHOLD << 1;
|
||||
Set<String> valuesSet = new HashSet<>();
|
||||
for (int i = 0; i < CARDINALITY; ++i) {
|
||||
final int length = TestUtil.nextInt(random(), 512, 1024);
|
||||
valuesSet.add(TestUtil.randomSimpleString(random(), length));
|
||||
}
|
||||
int valuesCount = valuesSet.size();
|
||||
List<String> values = new ArrayList<>(valuesSet);
|
||||
|
||||
try (Directory directory = newDirectory()) {
|
||||
Analyzer analyzer = new StandardAnalyzer();
|
||||
IndexWriterConfig config = new IndexWriterConfig(analyzer);
|
||||
config.setCodec(bestCompression);
|
||||
config.setUseCompoundFile(false);
|
||||
IndexWriter writer = new IndexWriter(directory, config);
|
||||
for (int i = 0; i < 256; i++) {
|
||||
Document doc = new Document();
|
||||
doc.add(new StringField("id", "Doc" + i, Field.Store.NO));
|
||||
doc.add(new SortedDocValuesField("sdv", new BytesRef(values.get(i % valuesCount))));
|
||||
writer.addDocument(doc);
|
||||
}
|
||||
writer.commit();
|
||||
writer.forceMerge(1);
|
||||
DirectoryReader ireader = DirectoryReader.open(writer);
|
||||
writer.close();
|
||||
|
||||
LeafReader reader = getOnlyLeafReader(ireader);
|
||||
// Check values count.
|
||||
SortedDocValues ssdvMulti = reader.getSortedDocValues("sdv");
|
||||
assertEquals(valuesCount, ssdvMulti.getValueCount());
|
||||
ireader.close();
|
||||
}
|
||||
}
|
||||
|
||||
// Ensure the old segment can be merged together with the new compressed segment.
|
||||
public void testMergeWithUncompressedSegment() throws IOException {
|
||||
final int CARDINALITY = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_COMPRESSION_THRESHOLD << 1;
|
||||
Set<String> valuesSet = new HashSet<>();
|
||||
for (int i = 0; i < CARDINALITY; ++i) {
|
||||
final int length = TestUtil.nextInt(random(), 10, 30);
|
||||
// Add common suffix for better compression ratio.
|
||||
valuesSet.add(TestUtil.randomSimpleString(random(), length));
|
||||
}
|
||||
List<String> values = new ArrayList<>(valuesSet);
|
||||
int valuesCount = values.size();
|
||||
|
||||
try (Directory directory = newDirectory()) {
|
||||
// 1. Write 256 documents without terms dict compression.
|
||||
Analyzer analyzer = new StandardAnalyzer();
|
||||
IndexWriterConfig config = new IndexWriterConfig(analyzer);
|
||||
config.setCodec(bestSpeed);
|
||||
config.setUseCompoundFile(false);
|
||||
IndexWriter writer = new IndexWriter(directory, config);
|
||||
for (int i = 0; i < 256; i++) {
|
||||
Document doc = new Document();
|
||||
doc.add(new StringField("id", "Doc" + i, Field.Store.NO));
|
||||
doc.add(new SortedSetDocValuesField("ssdv", new BytesRef(values.get(i % valuesCount))));
|
||||
doc.add(
|
||||
new SortedSetDocValuesField("ssdv", new BytesRef(values.get((i + 1) % valuesCount))));
|
||||
doc.add(
|
||||
new SortedSetDocValuesField("ssdv", new BytesRef(values.get((i + 2) % valuesCount))));
|
||||
doc.add(new SortedDocValuesField("sdv", new BytesRef(values.get(i % valuesCount))));
|
||||
writer.addDocument(doc);
|
||||
}
|
||||
writer.commit();
|
||||
DirectoryReader ireader = DirectoryReader.open(writer);
|
||||
assertEquals(256, ireader.numDocs());
|
||||
LeafReader reader = getOnlyLeafReader(ireader);
|
||||
SortedSetDocValues ssdv = reader.getSortedSetDocValues("ssdv");
|
||||
assertEquals(valuesCount, ssdv.getValueCount());
|
||||
SortedDocValues sdv = reader.getSortedDocValues("sdv");
|
||||
assertEquals(valuesCount, sdv.getValueCount());
|
||||
ireader.close();
|
||||
writer.close();
|
||||
|
||||
// 2. Add another 100 documents, and enabling terms dict compression.
|
||||
config = new IndexWriterConfig(analyzer);
|
||||
config.setCodec(bestCompression);
|
||||
config.setUseCompoundFile(false);
|
||||
writer = new IndexWriter(directory, config);
|
||||
// Add 2 new values.
|
||||
valuesSet.add(TestUtil.randomSimpleString(random(), 10));
|
||||
valuesSet.add(TestUtil.randomSimpleString(random(), 10));
|
||||
values = new ArrayList<>(valuesSet);
|
||||
valuesCount = valuesSet.size();
|
||||
|
||||
for (int i = 256; i < 356; i++) {
|
||||
Document doc = new Document();
|
||||
doc.add(new StringField("id", "Doc" + i, Field.Store.NO));
|
||||
doc.add(new SortedSetDocValuesField("ssdv", new BytesRef(values.get(i % valuesCount))));
|
||||
doc.add(new SortedDocValuesField("sdv", new BytesRef(values.get(i % valuesCount))));
|
||||
writer.addDocument(doc);
|
||||
}
|
||||
writer.commit();
|
||||
writer.forceMerge(1);
|
||||
ireader = DirectoryReader.open(writer);
|
||||
assertEquals(356, ireader.numDocs());
|
||||
reader = getOnlyLeafReader(ireader);
|
||||
ssdv = reader.getSortedSetDocValues("ssdv");
|
||||
assertEquals(valuesCount, ssdv.getValueCount());
|
||||
ireader.close();
|
||||
writer.close();
|
||||
}
|
||||
}
|
||||
|
||||
private static long writeAndGetDocValueFileSize(Codec codec, List<String> values)
|
||||
throws IOException {
|
||||
int valuesCount = values.size();
|
||||
long dvdFileSize = -1;
|
||||
try (Directory directory = newDirectory()) {
|
||||
Analyzer analyzer = new StandardAnalyzer();
|
||||
IndexWriterConfig config = new IndexWriterConfig(analyzer);
|
||||
config.setCodec(codec);
|
||||
config.setUseCompoundFile(false);
|
||||
IndexWriter writer = new IndexWriter(directory, config);
|
||||
for (int i = 0; i < 256; i++) {
|
||||
Document doc = new Document();
|
||||
doc.add(new StringField("id", "Doc" + i, Field.Store.NO));
|
||||
// Multi value sorted-set field.
|
||||
doc.add(
|
||||
new SortedSetDocValuesField("ssdv_multi_", new BytesRef(values.get(i % valuesCount))));
|
||||
doc.add(
|
||||
new SortedSetDocValuesField(
|
||||
"ssdv_multi_", new BytesRef(values.get((i + 1) % valuesCount))));
|
||||
doc.add(
|
||||
new SortedSetDocValuesField(
|
||||
"ssdv_multi_", new BytesRef(values.get((i + 2) % valuesCount))));
|
||||
// Single value sorted-set field.
|
||||
doc.add(
|
||||
new SortedSetDocValuesField("ssdv_single_", new BytesRef(values.get(i % valuesCount))));
|
||||
// Sorted field.
|
||||
doc.add(new SortedDocValuesField("sdv", new BytesRef(values.get(i % valuesCount))));
|
||||
writer.addDocument(doc);
|
||||
}
|
||||
writer.commit();
|
||||
writer.forceMerge(1);
|
||||
DirectoryReader ireader = DirectoryReader.open(writer);
|
||||
writer.close();
|
||||
|
||||
LeafReader reader = getOnlyLeafReader(ireader);
|
||||
// Check values count.
|
||||
SortedSetDocValues ssdvMulti = reader.getSortedSetDocValues("ssdv_multi_");
|
||||
assertEquals(valuesCount, ssdvMulti.getValueCount());
|
||||
for (int i = 0; i < valuesCount; i++) {
|
||||
BytesRef term = ssdvMulti.lookupOrd(i);
|
||||
assertTrue(term.bytes.length > 0);
|
||||
}
|
||||
for (int i = 0; i < valuesCount; i++) {
|
||||
for (int j = 0; j < 3; j++) {
|
||||
assertTrue(ssdvMulti.lookupTerm(new BytesRef(values.get((i + j) % valuesCount))) >= 0);
|
||||
}
|
||||
}
|
||||
|
||||
SortedSetDocValues ssdvSingle = reader.getSortedSetDocValues("ssdv_single_");
|
||||
assertEquals(valuesCount, ssdvSingle.getValueCount());
|
||||
for (int i = 0; i < valuesCount; i++) {
|
||||
assertTrue(ssdvSingle.lookupTerm(new BytesRef(values.get(i % valuesCount))) >= 0);
|
||||
}
|
||||
|
||||
SortedDocValues sdv = reader.getSortedDocValues("sdv");
|
||||
assertEquals(valuesCount, sdv.getValueCount());
|
||||
for (int i = 0; i < valuesCount; i++) {
|
||||
assertTrue(sdv.lookupTerm(new BytesRef(values.get(i % valuesCount))) >= 0);
|
||||
}
|
||||
|
||||
dvdFileSize = docValueFileSize(directory);
|
||||
assertTrue(dvdFileSize > 0);
|
||||
ireader.close();
|
||||
}
|
||||
|
||||
return dvdFileSize;
|
||||
}
|
||||
|
||||
static long docValueFileSize(Directory d) throws IOException {
|
||||
for (String file : d.listAll()) {
|
||||
if (file.endsWith(Lucene90DocValuesFormat.DATA_EXTENSION)) {
|
||||
return d.fileLength(file);
|
||||
}
|
||||
}
|
||||
return -1;
|
||||
}
|
||||
}
|
|
@ -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;
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
Loading…
Reference in New Issue