This commit is contained in:
Karl Wright 2018-12-11 09:30:17 -05:00
commit 874937aba8
12 changed files with 163 additions and 1785 deletions

View File

@ -220,9 +220,6 @@ Optimizations
to early terminate the iterator if the minimum score is greater than the constant
score. (Christophe Bismuth via Jim Ferenczi)
* LUCENE-8374: Reduce reads for sparse DocValues and whole number numeric DocValues.
(Toke Eskildsen)
======================= Lucene 7.7.0 =======================
Build

View File

@ -50,9 +50,6 @@ import org.apache.lucene.util.RoaringDocIdSet;
final class IndexedDISI extends DocIdSetIterator {
static final int MAX_ARRAY_LENGTH = (1 << 12) - 1;
static final String NO_NAME = "n/a";
public final String name;
private static void flush(int block, FixedBitSet buffer, int cardinality, IndexOutput out) throws IOException {
assert block >= 0 && block < 65536;
@ -101,49 +98,19 @@ final class IndexedDISI extends DocIdSetIterator {
/** The slice that stores the {@link DocIdSetIterator}. */
private final IndexInput slice;
private final long cost;
private final IndexedDISICache cache;
IndexedDISI(IndexInput in, long offset, long length, long cost) throws IOException {
this(in, offset, length, cost, NO_NAME);
}
IndexedDISI(IndexInput in, long offset, long length, long cost, String name) throws IOException {
this(in, offset, length, cost, null, name);
}
IndexedDISI(IndexInput in, long offset, long length, long cost, IndexedDISICache cache) throws IOException {
this(in, offset, length, cost, cache, NO_NAME);
}
IndexedDISI(IndexInput in, long offset, long length, long cost, IndexedDISICache cache, String name) throws IOException {
this(in.slice("docs", offset, length), cost, cache, name);
this(in.slice("docs", offset, length), cost);
}
// This constructor allows to pass the slice directly in case it helps reuse
// see eg. Lucene70 norms producer's merge instance
IndexedDISI(IndexInput slice, long cost) throws IOException {
this(slice, cost, NO_NAME);
}
// This constructor allows to pass the slice directly in case it helps reuse
// see eg. Lucene70 norms producer's merge instance
IndexedDISI(IndexInput slice, long cost, String name) throws IOException {
this(slice, cost, null, name);
// IndexedDISICacheFactory.debug(
// "Non-cached direct slice IndexedDISI with length " + slice.length() + ": " + slice.toString());
}
IndexedDISI(IndexInput slice, long cost, IndexedDISICache cache) throws IOException {
this(slice, cost, cache, NO_NAME);
}
// This constructor allows to pass the slice directly in case it helps reuse
// see eg. Lucene70 norms producer's merge instance
IndexedDISI(IndexInput slice, long cost, IndexedDISICache cache, String name) {
this.name = name;
this.slice = slice;
this.cost = cost;
this.cache = cache == null ? IndexedDISICache.EMPTY : cache;
}
private int block = -1;
private long blockStart; // Used with the DENSE cache
private long blockEnd;
private int nextBlockIndex = -1;
Method method;
@ -159,8 +126,6 @@ final class IndexedDISI extends DocIdSetIterator {
private int wordIndex = -1;
// number of one bits encountered so far, including those of `word`
private int numberOfOnes;
// Used with rank for jumps inside of DENSE
private int denseOrigoIndex;
// ALL variables
private int gap;
@ -173,7 +138,6 @@ final class IndexedDISI extends DocIdSetIterator {
@Override
public int advance(int target) throws IOException {
final int targetBlock = target & 0xFFFF0000;
// Note: The cache makes it easy to add support for random access. This has not been done as the API forbids it
if (block < targetBlock) {
advanceBlock(targetBlock);
}
@ -199,20 +163,6 @@ final class IndexedDISI extends DocIdSetIterator {
}
private void advanceBlock(int targetBlock) throws IOException {
if (targetBlock >= block+2) { // 1 block skip is (slightly) faster to do without block jump table
long offset = cache.getFilePointerForBlock(targetBlock >> IndexedDISICache.BLOCK_BITS);
if (offset != -1 && offset > slice.getFilePointer()) {
int origo = cache.getIndexForBlock(targetBlock >> IndexedDISICache.BLOCK_BITS);
if (origo != -1) {
this.nextBlockIndex = origo - 1; // -1 to compensate for the always-added 1 in readBlockHeader
slice.seek(offset);
readBlockHeader();
return;
}
}
}
// Fallback to non-cached
do {
slice.seek(blockEnd);
readBlockHeader();
@ -220,7 +170,6 @@ final class IndexedDISI extends DocIdSetIterator {
}
private void readBlockHeader() throws IOException {
blockStart = slice.getFilePointer();
block = Short.toUnsignedInt(slice.readShort()) << 16;
assert block >= 0;
final int numValues = 1 + Short.toUnsignedInt(slice.readShort());
@ -238,7 +187,6 @@ final class IndexedDISI extends DocIdSetIterator {
blockEnd = slice.getFilePointer() + (1 << 13);
wordIndex = -1;
numberOfOnes = index + 1;
denseOrigoIndex = numberOfOnes;
}
}
@ -302,10 +250,6 @@ final class IndexedDISI extends DocIdSetIterator {
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
disi.rankSkip(disi, target);
for (int i = disi.wordIndex + 1; i <= targetWordIndex; ++i) {
disi.word = disi.slice.readLong();
disi.numberOfOnes += Long.bitCount(disi.word);
@ -319,10 +263,7 @@ final class IndexedDISI extends DocIdSetIterator {
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;
@ -331,18 +272,12 @@ final class IndexedDISI extends DocIdSetIterator {
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
disi.rankSkip(disi, target);
for (int i = disi.wordIndex + 1; i <= targetWordIndex; ++i) {
disi.word = disi.slice.readLong();
disi.numberOfOnes += Long.bitCount(disi.word);
@ -353,8 +288,6 @@ final class IndexedDISI extends DocIdSetIterator {
disi.index = disi.numberOfOnes - Long.bitCount(leftBits);
return (leftBits & 1L) != 0;
}
},
ALL {
@Override
@ -379,47 +312,4 @@ final class IndexedDISI extends DocIdSetIterator {
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 target the wanted docID for which to calculate set-flag and index.
* @throws IOException if a disi seek failed.
*/
private void rankSkip(IndexedDISI disi, int target) throws IOException {
final int targetInBlock = target & 0xFFFF;
final int targetWordIndex = targetInBlock >>> 6;
// If the distance between the current position and the target is >= 8
// then it pays to use the rank to jump
if (!(disi.cache.hasRank() && targetWordIndex - disi.wordIndex >= IndexedDISICache.RANK_BLOCK_LONGS)) {
return;
}
int rankPos = disi.cache.denseRankPosition(target);
if (rankPos == -1) {
return;
}
int rank = disi.cache.getRankInBlock(rankPos);
if (rank == -1) {
return;
}
int rankIndex = disi.denseOrigoIndex + rank;
int rankWordIndex = (rankPos & 0xFFFF) >> 6;
long rankOffset = disi.blockStart + 4 + (rankWordIndex * 8);
long mark = disi.slice.getFilePointer();
disi.slice.seek(rankOffset);
long rankWord = disi.slice.readLong();
int rankNOO = rankIndex + Long.bitCount(rankWord);
rankOffset += Long.BYTES;
//disi.slice.seek(mark);
disi.wordIndex = rankWordIndex;
disi.word = rankWord;
disi.numberOfOnes = rankNOO;
}
}

View File

@ -1,330 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene70;
import java.io.IOException;
import java.util.Arrays;
import java.util.Locale;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.packed.PackedInts;
import static org.apache.lucene.codecs.lucene70.IndexedDISI.MAX_ARRAY_LENGTH;
/**
* Caching of IndexedDISI with two strategies:
*
* A lookup table for block blockCache and index, and a rank structure for DENSE block lookups.
*
* The lookup table is an array of {@code long}s with an entry for each block. It allows for
* direct jumping to the block, as opposed to iteration from the current position and forward
* one block at a time.
*
* Each long entry consists of 2 logical parts:
*
* The first 31 bits holds the index (number of set bits in the blocks) up to just before the
* wanted block. The next 33 bits holds the offset 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^17 bits to avoid overflow. This is currently the case, with the largest
* block being DENSE and using 2^16 + 32 bits, and is likely to continue to hold as using
* more than double the amount of bits is unlikely to be an efficient representation.
* The cache overhead is numDocs/1024 bytes.
*
* Note: There are 4 types of blocks: ALL, DENSE, SPARSE and non-existing (0 set bits).
* In the case of non-existing blocks, the entry in the lookup table has index equal to the
* previous entry and offset equal to the next non-empty block.
*
*
* The rank structure for DENSE blocks is an array of unsigned {@code short}s with an entry
* for each sub-block of 512 bits out of the 65536 bits in the outer block.
*
* Each rank-entry states the number of set bits within the block up to the bit before the
* bit positioned at the start of the sub-block.
* Note that that the rank entry of the first sub-block is always 0 and that the last entry can
* at most be 65536-512 = 65024 and thus will always fit into an unsigned short.
*
* See https://en.wikipedia.org/wiki/Succinct_data_structure for details on rank structures.
* The alternative to using the rank structure is iteration and summing of set bits for all
* entries in the DENSE sub-block up until the wanted bit, with a worst-case of 1024 entries.
* The rank cache overhead for a single DENSE block is 128 shorts (128*16 = 2048 bits) or
* 1/32th.
*
* The ranks for the DENSE blocks are stored in a structure shared for the whole array of
* blocks, DENSE or not. To avoid overhead that structure is itself sparse. See
* {@link LongCompressor} for details on DENSE structure sparseness.
*
*
* The performance overhead for creating a cache instance is equivalent to accessing all
* DocValues values for the given field, i.e. it scales lineary to field size. On modern
* hardware it is in the ballpark of 1ms for 5M values on modern hardware. Caveat lector:
* At the point of writing, performance points are only available for 2 real-world setups.
*/
public class IndexedDISICache implements Accountable {
private static final int BLOCK = 65536; // The number of docIDs that a single block represents
static final int BLOCK_BITS = 16;
private static final long BLOCK_INDEX_SHIFT = 33; // Number of bits to shift a lookup entry to get the index
private static final long BLOCK_INDEX_MASK = ~0L << BLOCK_INDEX_SHIFT; // The index bits in a lookup entry
private static final long BLOCK_LOOKUP_MASK = ~BLOCK_INDEX_MASK; // The offset bits in a lookup entry
private static final int RANK_BLOCK = 512; // The number of docIDs/bits in each rank-sub-block within a DENSE block
static final int RANK_BLOCK_LONGS = 512/Long.SIZE; // The number of longs making up a rank-block (8)
private static final int RANK_BLOCK_BITS = 9;
private static final int RANKS_PER_BLOCK = BLOCK/RANK_BLOCK;
private PackedInts.Reader rank; // One every 512 docs, sparsely represented as not all blocks are DENSE
private long[] blockCache = null; // One every 65536 docs, contains index & slice position
private String creationStats = "";
private final String name; // Identifier for debug, log & inspection
// Flags for not-yet-defined-values used during building
private static final long BLOCK_EMPTY_INDEX = ~0L << BLOCK_INDEX_SHIFT;
private static final long BLOCK_EMPTY_LOOKUP = BLOCK_LOOKUP_MASK;
private static final long BLOCK_EMPTY = BLOCK_EMPTY_INDEX | BLOCK_EMPTY_LOOKUP;
/**
* Builds the stated caches for the given IndexInput.
*
* @param in positioned at the start of the logical underlying bitmap.
*/
IndexedDISICache(IndexInput in, String name) throws IOException {
blockCache = new long[16]; // Will be extended when needed
Arrays.fill(blockCache, BLOCK_EMPTY);
this.name = name;
updateCaches(in);
}
private IndexedDISICache() {
this.blockCache = null;
this.rank = null;
this.name = "";
}
/**
* Shared between all structures that are too small to meaningfully use jump-tables.
*/
public static final IndexedDISICache EMPTY = new IndexedDISICache();
/**
* If available, returns a position within the underlying {@link IndexInput} for the start of the block
* containing the wanted bit (the target) or the next non-EMPTY block, if the block representing the bit is empty.
* @param targetBlock the index for the block to resolve (docID / 65536).
* @return the offset for the block for target or -1 if it cannot be resolved.
*/
long getFilePointerForBlock(int targetBlock) {
long offset = blockCache == null || blockCache.length <= targetBlock ?
-1 : blockCache[targetBlock] & BLOCK_LOOKUP_MASK;
return offset == BLOCK_EMPTY_LOOKUP ? -1 : offset;
}
/**
* If available, returns the index; number of set bits before the wanted block.
* @param targetBlock the block to resolve (docID / 65536).
* @return the index for the block or -1 if it cannot be resolved.
*/
int getIndexForBlock(int targetBlock) {
if (blockCache == null || blockCache.length <= targetBlock) {
return -1;
}
return (blockCache[targetBlock] & BLOCK_INDEX_MASK) == BLOCK_EMPTY_INDEX ?
-1 : (int)(blockCache[targetBlock] >>> BLOCK_INDEX_SHIFT);
}
/**
* Given a target (docID), this method returns the first docID in the entry containing the target.
* @param target the docID for which an index is wanted.
* @return the docID where the rank is known. This will be lte target.
*/
// TODO: This method requires a lot of knowledge of the intrinsics of the cache. Usage should be simplified
int denseRankPosition(int target) {
return target >> RANK_BLOCK_BITS << RANK_BLOCK_BITS;
}
/**
* Offsets stated the block starts.
* @return true if the cache has offsets.
*/
public boolean hasOffsets() {
return blockCache != null;
}
boolean hasRank() {
return rank != null;
}
/**
* Get the rank (index) for all set bits up to just before the given rankPosition in the block.
* The caller is responsible for deriving the count of bits up to the docID target from the rankPosition.
* The caller is also responsible for keeping track of set bits up to the current block.
* Important: This only accepts rankPositions that aligns to {@link #RANK_BLOCK} boundaries.
* Note 1: Use {@link #denseRankPosition(int)} to obtain a calid rankPosition for a wanted docID.
* Note 2: The caller should seek to the rankPosition in the underlying slice to keep everything in sync.
* @param rankPosition a docID target that aligns to {@link #RANK_BLOCK}.
* @return the rank (index / set bits count) up to just before the given rankPosition.
* If rank is disabled, -1 is returned.
*/
// TODO: This method requires a lot of knowledge of the intrinsics of the cache. Usage should be simplified
int getRankInBlock(int rankPosition) {
if (rank == null) {
return -1;
}
assert rankPosition == denseRankPosition(rankPosition);
int rankIndex = rankPosition >> RANK_BLOCK_BITS;
return rankIndex >= rank.size() ? -1 : (int) rank.get(rankIndex);
}
private void updateCaches(IndexInput slice) throws IOException {
final long startOffset = slice.getFilePointer();
final long startTime = System.nanoTime();
AtomicInteger statBlockALL = new AtomicInteger(0);
AtomicInteger statBlockDENSE = new AtomicInteger(0);
AtomicInteger statBlockSPARSE = new AtomicInteger(0);
// Fill phase
int largestBlock = fillCache(slice, statBlockALL, statBlockDENSE, statBlockSPARSE);
freezeCaches(largestBlock);
slice.seek(startOffset); // Leave it as we found it
creationStats = String.format(Locale.ENGLISH,
"name=%s, blocks=%d (ALL=%d, DENSE=%d, SPARSE=%d, EMPTY=%d), time=%dms, block=%d bytes, rank=%d bytes",
name,
largestBlock+1, statBlockALL.get(), statBlockDENSE.get(), statBlockSPARSE.get(),
(largestBlock+1-statBlockALL.get()-statBlockDENSE.get()-statBlockSPARSE.get()),
(System.nanoTime()-startTime)/1000000,
blockCache == null ? 0 : blockCache.length*Long.BYTES,
rank == null ? 0 : rank.ramBytesUsed());
}
private int fillCache(
IndexInput slice, AtomicInteger statBlockALL, AtomicInteger statBlockDENSE, AtomicInteger statBlockSPARSE)
throws IOException {
char[] buildRank = new char[256];
int largestBlock = -1;
long index = 0;
int rankIndex = -1;
while (slice.getFilePointer() < slice.length()) {
final long startFilePointer = slice.getFilePointer();
final int blockIndex = Short.toUnsignedInt(slice.readShort());
final int numValues = 1 + Short.toUnsignedInt(slice.readShort());
assert blockIndex > largestBlock;
if (blockIndex == DocIdSetIterator.NO_MORE_DOCS >>> 16) { // End reached
assert Short.toUnsignedInt(slice.readShort()) == (DocIdSetIterator.NO_MORE_DOCS & 0xFFFF);
break;
}
largestBlock = blockIndex;
blockCache = ArrayUtil.grow(blockCache, blockIndex+1); // No-op if large enough
blockCache[blockIndex] = (index << BLOCK_INDEX_SHIFT) | startFilePointer;
index += numValues;
if (numValues <= MAX_ARRAY_LENGTH) { // SPARSE
statBlockSPARSE.incrementAndGet();
slice.seek(slice.getFilePointer() + (numValues << 1));
continue;
}
if (numValues == 65536) { // ALL
statBlockALL.incrementAndGet();
// Already at next block offset
continue;
}
// The block is DENSE
statBlockDENSE.incrementAndGet();
long nextBlockOffset = slice.getFilePointer() + (1 << 13);
int setBits = 0;
int rankOrigo = blockIndex << 16 >> 9; // Double shift for clarity: The compiler will simplify it
for (int rankDelta = 0 ; rankDelta < RANKS_PER_BLOCK ; rankDelta++) { // 128 rank-entries in a block
rankIndex = rankOrigo + rankDelta;
buildRank = ArrayUtil.grow(buildRank, rankIndex+1);
buildRank[rankIndex] = (char)setBits;
for (int i = 0 ; i < 512/64 ; i++) { // 8 longs for each rank-entry
setBits += Long.bitCount(slice.readLong());
}
}
assert slice.getFilePointer() == nextBlockOffset;
}
// Compress the buildRank as it is potentially very sparse
if (rankIndex < 0) {
rank = null;
} else {
PackedInts.Mutable ranks = PackedInts.getMutable(rankIndex, 16, PackedInts.DEFAULT); // Char = 16 bit
for (int i = 0 ; i < rankIndex ; i++) {
ranks.set(i, buildRank[i]);
}
rank = LongCompressor.compress(ranks);
}
return largestBlock;
}
private void freezeCaches(int largestBlock) {
if (largestBlock == -1) { // No set bit: Disable the caches
blockCache = null;
rank = null;
return;
}
// Reduce size to minimum
if (blockCache.length-1 > largestBlock) {
long[] newBC = new long[Math.max(largestBlock - 1, 1)];
System.arraycopy(blockCache, 0, newBC, 0, newBC.length);
blockCache = newBC;
}
// Set non-defined blockCache entries (caused by blocks with 0 set bits) to the subsequently defined one
long latest = BLOCK_EMPTY;
for (int i = blockCache.length-1; i >= 0 ; i--) {
long current = blockCache[i];
if (current == BLOCK_EMPTY) {
blockCache[i] = latest;
} else {
latest = current;
}
}
}
/**
* Creation stats intended for human inspection.
* @return Human readable details from the creation of the cache instance.
*/
public String getCreationStats() {
return creationStats;
}
/**
* Cache name, as stated in the constructor.
* @return Human-readable name for the cache instance.
*/
public String getName() {
return name;
}
@Override
public long ramBytesUsed() {
return (blockCache == null ? 0 : RamUsageEstimator.sizeOf(blockCache)) +
(rank == null ? 0 : rank.ramBytesUsed()) +
RamUsageEstimator.NUM_BYTES_OBJECT_REF*3 +
RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + creationStats.length()*2;
}
}

View File

@ -1,260 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene70;
import java.io.IOException;
import java.util.HashMap;
import java.util.Locale;
import java.util.Map;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.RandomAccessInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.RamUsageEstimator;
/**
* Creates and stores caches for {@link IndexedDISI} and {@link Lucene70DocValuesProducer}.
* The caches are stored in maps, where the key is made up from offset and length of a slice
* in an underlying segment. Each segment uses their own IndexedDISICacheFactory.
*
* See {@link IndexedDISICache} for details on the caching.
*/
public class IndexedDISICacheFactory implements Accountable {
/**
* If the slice with the DISI-data is less than this number of bytes, don't create a cache.
* This is a very low number as the DISI-structure very efficiently represents EMPTY and ALL blocks.
*/
private static int MIN_LENGTH_FOR_CACHING = 50; // Set this very low: Could be 9 EMPTY followed by a SPARSE
// jump-table and rank for DISI blocks
private final Map<Long, IndexedDISICache> disiPool = new HashMap<>();
// jump-table for numerics with variable bits per value (dates, longs...)
private final Map<String, VaryingBPVJumpTable> vBPVPool = new HashMap<>();
/**
* Creates a {@link IndexedDISICache} and {@link VaryingBPVJumpTable} holding factory,
* intended for shared use within a single segment.
*/
public IndexedDISICacheFactory() { }
/**
* Create a cached {@link IndexedDISI} instance.
* @param data persistent data containing the DISI-structure.
* @param cost cost as defined for IndexedDISI.
* @param name identifier for the DISI-structure for debug purposes.
* @return a cached IndexedDISI or a plain IndexedDISI, if caching is not applicable.
* @throws IOException if the DISI-structure could not be accessed.
*/
IndexedDISI createCachedIndexedDISI(IndexInput data, long key, int cost, String name) throws IOException {
IndexedDISICache cache = getCache(data, key, name);
return new IndexedDISI(data, cost, cache, name);
}
/**
* Create a cached {@link IndexedDISI} instance.
* @param data persistent data containing the DISI-structure.
* @param offset same as the offset that will also be used for creating an {@link IndexedDISI}.
* @param length same af the length that will also be used for creating an {@link IndexedDISI}.
* @param cost cost as defined for IndexedDISI.
* @param name identifier for the DISI-structure for debug purposes.
* @return a cached IndexedDISI or a plain IndexedDISI, if caching is not applicable.
* @throws IOException if the DISI-structure could not be accessed.
*/
IndexedDISI createCachedIndexedDISI(IndexInput data, long offset, long length, long cost, String name)
throws IOException {
IndexedDISICache cache = getCache(data, offset, length, name);
return new IndexedDISI(data, offset, length, cost, cache, name);
}
/**
* Creates a cache (jump table) for variable bits per value numerics and returns it.
* If the cache has previously been created, the old cache is returned.
* @param name the name for the cache, typically the field name. Used as key for later retrieval.
* @param slice the long values with varying bits per value.
* @param valuesLength the length in bytes of the slice.
* @return a jump table for the longs in the given slice or null if the structure is not suitable for caching.
*/
VaryingBPVJumpTable getVBPVJumpTable(String name, RandomAccessInput slice, long valuesLength) throws IOException {
VaryingBPVJumpTable jumpTable = vBPVPool.get(name);
if (jumpTable == null) {
// TODO: Avoid overlapping builds of the same jump table for performance reasons
jumpTable = new VaryingBPVJumpTable(slice, name, valuesLength);
vBPVPool.put(name, jumpTable);
}
return jumpTable;
}
/**
* Creates a cache (jump table) for {@link IndexedDISI}.
* If the cache has previously been created, the old cache is returned.
* @param data the slice to create a cache for.
* @param offset same as the offset that will also be used for creating an {@link IndexedDISI}.
* @param length same af the length that will also be used for creating an {@link IndexedDISI}.
* @param name human readable designation, typically a field name. Used for debug, log and inspection.
* @return a cache for the given slice+offset+length or null if not suitable for caching.
*/
public IndexedDISICache getCache(IndexInput data, long offset, long length, String name) throws IOException {
if (length < MIN_LENGTH_FOR_CACHING) {
return null;
}
long key = offset + length;
IndexedDISICache cache = disiPool.get(key);
if (cache == null) {
// TODO: Avoid overlapping builds of the same cache for performance reason
cache = new IndexedDISICache(data.slice("docs", offset, length), name);
disiPool.put(key, cache);
}
return cache;
}
/**
* Creates a cache (jump table) for {@link IndexedDISI}.
* If the cache has previously been created, the old cache is returned.
* @param slice the input slice.
* @param key identifier for the cache, unique within the segment that originated the slice.
* Recommendation is offset+length for the slice, relative to the data mapping the segment.
* Warning: Do not use slice.getFilePointer and slice.length as they are not guaranteed
* to be unique within the segment (slice.getFilePointer is 0 when a sub-slice is created).
* @param name human readable designation, typically a field name. Used for debug, log and inspection.
* @return a cache for the given slice+offset+length or null if not suitable for caching.
*/
public IndexedDISICache getCache(IndexInput slice, long key, String name) throws IOException {
final long length = slice.length();
if (length < MIN_LENGTH_FOR_CACHING) {
return null;
}
IndexedDISICache cache = disiPool.get(key);
if (cache == null) {
// TODO: Avoid overlapping builds of the same cache
// Both BLOCK & DENSE caches are created as they might be requested later for the field,
// regardless of whether they are requested now
cache = new IndexedDISICache(slice, name);
disiPool.put(key, cache);
}
return cache;
}
/**
* Cache statistics intended for external inspection.
* @return the number of total blocks where jumps are accelerated by jump-tables.
*/
public long getDISIBlocksWithOffsetsCount() {
return disiPool.values().stream().filter(IndexedDISICache::hasOffsets).count();
}
/**
* Cache statistics intended for external inspection.
* @return the total number of DENSE blocks where index-counts are accelerated by rank.
*/
public long getDISIBlocksWithRankCount() {
return disiPool.values().stream().filter(IndexedDISICache::hasRank).count();
}
/**
* Cache statistics intended for external inspection.
* @return the number of numeric blocks where jumps are accelerated by jump-tables
*/
public long getVaryingBPVCount() {
return vBPVPool.size();
}
@Override
public long ramBytesUsed() {
long mem = RamUsageEstimator.shallowSizeOf(this) +
RamUsageEstimator.shallowSizeOf(disiPool) +
RamUsageEstimator.shallowSizeOf(vBPVPool);
for (Map.Entry<Long, IndexedDISICache> cacheEntry: disiPool.entrySet()) {
mem += RamUsageEstimator.shallowSizeOf(cacheEntry);
mem += RamUsageEstimator.sizeOf(cacheEntry.getKey());
mem += cacheEntry.getValue().ramBytesUsed();
}
for (Map.Entry<String, VaryingBPVJumpTable> cacheEntry: vBPVPool.entrySet()) {
String key = cacheEntry.getKey();
mem += RamUsageEstimator.shallowSizeOf(cacheEntry);
mem += RamUsageEstimator.shallowSizeOf(key)+key.length()*2;
mem += cacheEntry.getValue().ramBytesUsed();
}
return mem;
}
/**
* Releases all caches.
*/
void releaseAll() {
disiPool.clear();
vBPVPool.clear();
}
/**
* Jump table used by Lucene70DocValuesProducer.VaryingBPVReader to avoid iterating all blocks from
* current to wanted index for numerics with variable bits per value. The jump table holds offsets for all blocks.
*/
public static class VaryingBPVJumpTable implements Accountable {
// TODO: It is way overkill to use longs here for practically all indexes. Maybe a PackedInts representation?
long[] offsets = new long[10];
final String creationStats;
VaryingBPVJumpTable(RandomAccessInput slice, String name, long valuesLength) throws IOException {
final long startTime = System.nanoTime();
int block = -1;
long offset;
long blockEndOffset = 0;
int bitsPerValue;
do {
offset = blockEndOffset;
offsets = ArrayUtil.grow(offsets, block+2); // No-op if large enough
offsets[block+1] = offset;
bitsPerValue = slice.readByte(offset++);
offset += Long.BYTES; // Skip over delta as we do not resolve the values themselves at this point
if (bitsPerValue == 0) {
blockEndOffset = offset;
} else {
final int length = slice.readInt(offset);
offset += Integer.BYTES;
blockEndOffset = offset + length;
}
block++;
} while (blockEndOffset < valuesLength-Byte.BYTES);
offsets = ArrayUtil.copyOfSubArray(offsets, 0, block+1);
creationStats = String.format(Locale.ENGLISH,
"name=%s, blocks=%d, time=%dms",
name, offsets.length, (System.nanoTime()-startTime)/1000000);
}
/**
* @param block the logical block in the vBPV structure ( valueindex/16384 ).
* @return the index slice offset for the vBPV block (1 block = 16384 values) or -1 if not available.
*/
long getBlockOffset(long block) {
// Technically a limitation in caching vs. VaryingBPVReader to limit to 2b blocks of 16K values
return offsets[(int) block];
}
@Override
public long ramBytesUsed() {
return RamUsageEstimator.shallowSizeOf(this) +
(offsets == null ? 0 : RamUsageEstimator.sizeOf(offsets)) + // offsets
RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + creationStats.length()*2; // creationStats
}
}
}

View File

@ -1,251 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene70;
import org.apache.lucene.util.MathUtil;
import org.apache.lucene.util.RankBitSet;
import org.apache.lucene.util.packed.PackedInts;
/**
* Utility class for generating compressed read-only in-memory representations of longs.
* The representation is optimized towards random access primarily and space secondarily.
*
* The representation always applies delta-to-minvalue and greatest-common-divisor compression.
*
* Depending on the number of 0-entries and the length of the array, a sparse representation is
* used, using rank to improve access speed. Sparseness introduces an O(1) access time overhead.
* Sparseness can be turned off.
*/
public class LongCompressor {
/**
* The minimum amount of total values in the data set for sparse to be active.
*/
private static final int DEFAULT_MIN_TOTAL_VALUES_FOR_SPARSE = 10_000;
/**
* The minimum total amount of zero values in the data set for sparse to be active.
*/
private static final int DEFAULT_MIN_ZERO_VALUES_FOR_SPARSE = 500;
/**
* The minimum fraction of the data set that must be zero for sparse to be active.
*/
private static final double DEFAULT_MIN_ZERO_VALUES_FRACTION_FOR_SPARSE = 0.2; // 20% (just guessing of a value here)
/**
* LongCompressor exclusively uses static methods and is never instantiated.
*/
private LongCompressor() { }
/**
* Create a compact version of the given values.
* @param values PackedInts with no special constraints.
* @return a compact version of the given values or the given values if compression did not improve on heap overhead.
*/
public static PackedInts.Reader compress(PackedInts.Reader values) {
return compress(values, values.size());
}
/**
* Create a compact version of the given values from index 0 to length-1.
* @param values PackedInts with no special constraints.
* @param length the number of values to compress.
* @return a compact version of the given values or the given values if compression did not improve on heap overhead.
*/
public static PackedInts.Reader compress(PackedInts.Reader values, int length) {
return compress(values, length, true);
}
/**
* Create a compact version of the given values from index 0 to length-1.
* @param values PackedInts with no special constraints.
* @param length the number of values to compress.
* @param allowSparse if true and is the default limits matches the input, a sparse representation will be created.
* @return a compact version of the given values or the given values if compression did not improve on heap overhead.
*/
public static PackedInts.Reader compress(PackedInts.Reader values, int length, boolean allowSparse) {
return compress(values, length, allowSparse,
DEFAULT_MIN_TOTAL_VALUES_FOR_SPARSE,
DEFAULT_MIN_ZERO_VALUES_FOR_SPARSE,
DEFAULT_MIN_ZERO_VALUES_FRACTION_FOR_SPARSE);
}
/**
* Create a compact version of the given values from index 0 to length-1.
* @param values PackedInts with no special constraints.
* @param length the number of values to compress.
* @param allowSparse if true and is the default limits matches the input, a sparse representation will be created.
* @param minTotalSparse the minimum absolute number of 0-entries needed for a sparse representation.
* 0-entries are counted after minValue compression: {@code 3, 5, 3, 7, 16} has two 0-entries.
* @return a compact version of the given values or the given values if compression did not improve on heap overhead.
*/
public static PackedInts.Reader compress(
PackedInts.Reader values, int length, boolean allowSparse,
int minTotalSparse, int minZeroSparse, double minZeroFractionSparse) {
if (length == 0) {
return PackedInts.getMutable(0, 1, PackedInts.DEFAULT);
}
final long min = getMin(values, length);
final long gcd = getGCD(values, length, min);
final long maxCompressed = getMax(values, length, min, gcd);
int zeroCount;
if (!isPossiblySparseCandidate(length, allowSparse, minTotalSparse) ||
!isSparseCandidate(values, length, true, minTotalSparse,
(zeroCount = countZeroes(values, length, min, gcd)), minZeroSparse, minZeroFractionSparse)) {
// TODO: Add abort-early if it becomes apparent that no space saving is possible
PackedInts.Mutable inner =
PackedInts.getMutable(length, PackedInts.bitsRequired(maxCompressed), PackedInts.DEFAULT);
for (int i = 0 ; i < length ; i++) {
inner.set(i, (values.get(i)-min)/gcd);
}
PackedInts.Reader comp = new CompressedReader(inner, min, gcd);
// Sanity check that compression worked and if not, return the original input
return comp.ramBytesUsed() < values.ramBytesUsed() ? comp : values;
}
// Sparsify
RankBitSet rank = new RankBitSet(length);
PackedInts.Mutable inner =
PackedInts.getMutable(values.size()-zeroCount, PackedInts.bitsRequired(maxCompressed), PackedInts.DEFAULT);
int valueIndex = 0;
for (int i = 0 ; i < length ; i++) {
long value = (values.get(i)-min)/gcd;
if (value != 0) {
rank.set(i);
inner.set(valueIndex++, value);
}
}
rank.buildRankCache();
PackedInts.Reader comp = new CompressedReader(inner, min, gcd, rank);
// Sanity check that compression worked and if not, return the original input
return comp.ramBytesUsed() < values.ramBytesUsed() ? comp : values;
}
// Fast check
private static boolean isPossiblySparseCandidate(int length, boolean allowSparse, int minTotalSparse) {
return allowSparse && minTotalSparse <= length;
}
// Also fast, but requires zeroCount which is slow to calculate
private static boolean isSparseCandidate(
PackedInts.Reader values, int length, boolean allowSparse, int minTotalSparse,
int zeroCount, int minZeroSparse, double minZeroFractionSparse) {
return allowSparse && minTotalSparse <= length &&
minZeroSparse < zeroCount && minZeroFractionSparse < 1.0 * zeroCount / length;
}
// Not very fast as is requires #length divisions.
private static int countZeroes(PackedInts.Reader values, int length, long min, final long gcd) {
int zeroCount = 0;
for (int i = 0 ; i < length ; i++) {
if ((values.get(i)-min)/gcd == 0) { // Hope the case where gcd==1 gets JITted. We could add a switch to be sure?
zeroCount++;
}
}
return zeroCount;
}
static class CompressedReader extends PackedInts.Reader {
private final PackedInts.Reader inner;
final long min;
final long gcd;
final RankBitSet rank;
CompressedReader(PackedInts.Reader inner, long min, long gcd) {
this(inner, min, gcd, null);
}
CompressedReader(PackedInts.Reader inner, long min, long gcd, RankBitSet rank) {
this.inner = inner;
this.min = min;
this.gcd = gcd;
this.rank = rank;
}
@Override
public int size() {
return rank == null ? inner.size() : rank.length();
}
@Override
public long get(int docID) {
// No rank: The value at the index
// Rank but no set bit: min*gcd
// Rank and set bit: (The value at the rank + min) * gcd
return (rank == null ? inner.get(docID) : rank.get(docID) ? inner.get(rank.rank(docID)) : 0) * gcd + min;
}
@Override
public long ramBytesUsed() {
return inner.ramBytesUsed() + (rank == null ? 0 : rank.ramBytesUsed());
}
}
private static long getMin(PackedInts.Reader values, int length) {
long min = Long.MAX_VALUE;
for (int i = 0 ; i < length ; i++) {
if (min > values.get(i)) {
min = values.get(i);
}
}
return min;
}
// GCD-code takes & adjusted from Lucene70DocValuesConsumer
private static long getGCD(final PackedInts.Reader values, final int length, final long min) {
long gcd = -1;
for (int i = 0 ; i < length ; i++) {
long value = values.get(i)-min;
if (value == 0) {
continue;
}
if (gcd == -1) {
gcd = value;
continue;
}
if (value < Long.MIN_VALUE / 2 || value > Long.MAX_VALUE / 2) {
// in that case v - minValue might overflow and make the GCD computation return
// wrong results. Since these extreme values are unlikely, we just discard
// GCD computation for them
gcd = 1;
} else { // minValue needs to be set first
gcd = MathUtil.gcd(gcd, value);
}
if (gcd == 1) {
break;
}
}
return gcd == -1 ? 1 : gcd;
}
private static long getMax(final PackedInts.Reader values, final int length, final long min, final long gcd) {
long rawMax = Long.MIN_VALUE;
for (int i = 0 ; i < length ; i++) {
long value = values.get(i);
if (value > rawMax) {
rawMax = value;
}
}
return (rawMax-min)/gcd;
}
}

View File

@ -57,7 +57,6 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
private final Map<String,SortedNumericEntry> sortedNumerics = new HashMap<>();
private long ramBytesUsed;
private final IndexInput data;
private final IndexedDISICacheFactory disiCacheFactory = new IndexedDISICacheFactory();
private final int maxDoc;
/** expert: instantiates a new reader */
@ -120,23 +119,23 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
}
byte type = meta.readByte();
if (type == Lucene70DocValuesFormat.NUMERIC) {
numerics.put(info.name, readNumeric(meta, info.name));
numerics.put(info.name, readNumeric(meta));
} else if (type == Lucene70DocValuesFormat.BINARY) {
binaries.put(info.name, readBinary(meta, info.name));
binaries.put(info.name, readBinary(meta));
} else if (type == Lucene70DocValuesFormat.SORTED) {
sorted.put(info.name, readSorted(meta, info.name));
sorted.put(info.name, readSorted(meta));
} else if (type == Lucene70DocValuesFormat.SORTED_SET) {
sortedSets.put(info.name, readSortedSet(meta, info.name));
sortedSets.put(info.name, readSortedSet(meta));
} else if (type == Lucene70DocValuesFormat.SORTED_NUMERIC) {
sortedNumerics.put(info.name, readSortedNumeric(meta, info.name));
sortedNumerics.put(info.name, readSortedNumeric(meta));
} else {
throw new CorruptIndexException("invalid type: " + type, meta);
}
}
}
private NumericEntry readNumeric(ChecksumIndexInput meta, String name) throws IOException {
NumericEntry entry = new NumericEntry(name);
private NumericEntry readNumeric(ChecksumIndexInput meta) throws IOException {
NumericEntry entry = new NumericEntry();
readNumeric(meta, entry);
return entry;
}
@ -168,8 +167,8 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
entry.valuesLength = meta.readLong();
}
private BinaryEntry readBinary(ChecksumIndexInput meta, String name) throws IOException {
BinaryEntry entry = new BinaryEntry(name);
private BinaryEntry readBinary(ChecksumIndexInput meta) throws IOException {
BinaryEntry entry = new BinaryEntry();
entry.dataOffset = meta.readLong();
entry.dataLength = meta.readLong();
entry.docsWithFieldOffset = meta.readLong();
@ -187,8 +186,8 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
return entry;
}
private SortedEntry readSorted(ChecksumIndexInput meta, String name) throws IOException {
SortedEntry entry = new SortedEntry(name);
private SortedEntry readSorted(ChecksumIndexInput meta) throws IOException {
SortedEntry entry = new SortedEntry();
entry.docsWithFieldOffset = meta.readLong();
entry.docsWithFieldLength = meta.readLong();
entry.numDocsWithField = meta.readInt();
@ -199,12 +198,12 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
return entry;
}
private SortedSetEntry readSortedSet(ChecksumIndexInput meta, String name) throws IOException {
SortedSetEntry entry = new SortedSetEntry(name);
private SortedSetEntry readSortedSet(ChecksumIndexInput meta) throws IOException {
SortedSetEntry entry = new SortedSetEntry();
byte multiValued = meta.readByte();
switch (multiValued) {
case 0: // singlevalued
entry.singleValueEntry = readSorted(meta, name);
entry.singleValueEntry = readSorted(meta);
return entry;
case 1: // multivalued
break;
@ -246,8 +245,8 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
entry.termsIndexAddressesLength = meta.readLong();
}
private SortedNumericEntry readSortedNumeric(ChecksumIndexInput meta, String name) throws IOException {
SortedNumericEntry entry = new SortedNumericEntry(name);
private SortedNumericEntry readSortedNumeric(ChecksumIndexInput meta) throws IOException {
SortedNumericEntry entry = new SortedNumericEntry();
readNumeric(meta, entry);
entry.numDocsWithField = meta.readInt();
if (entry.numDocsWithField != entry.numValues) {
@ -263,23 +262,9 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
@Override
public void close() throws IOException {
data.close();
disiCacheFactory.releaseAll();
}
// Highly debatable if this is a sane construct as the name is only used for debug/logging/inspection purposes
// This was introduced in LUCENE-8374
private static class EntryImpl {
final String name;
public EntryImpl(String name) {
this.name = name;
}
}
private static class NumericEntry extends EntryImpl {
public NumericEntry(String name) {
super(name);
}
private static class NumericEntry {
long[] table;
int blockShift;
byte bitsPerValue;
@ -292,10 +277,7 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
long valuesLength;
}
private static class BinaryEntry extends EntryImpl {
public BinaryEntry(String name) {
super(name);
}
private static class BinaryEntry {
long dataOffset;
long dataLength;
long docsWithFieldOffset;
@ -308,10 +290,7 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
DirectMonotonicReader.Meta addressesMeta;
}
private static class TermsDictEntry extends EntryImpl {
public TermsDictEntry(String name) {
super(name);
}
private static class TermsDictEntry {
long termsDictSize;
int termsDictBlockShift;
DirectMonotonicReader.Meta termsAddressesMeta;
@ -329,9 +308,6 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
}
private static class SortedEntry extends TermsDictEntry {
public SortedEntry(String name) {
super(name);
}
long docsWithFieldOffset;
long docsWithFieldLength;
int numDocsWithField;
@ -341,9 +317,6 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
}
private static class SortedSetEntry extends TermsDictEntry {
public SortedSetEntry(String name) {
super(name);
}
SortedEntry singleValueEntry;
long docsWithFieldOffset;
long docsWithFieldLength;
@ -357,9 +330,6 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
}
private static class SortedNumericEntry extends NumericEntry {
public SortedNumericEntry(String name) {
super(name);
}
int numDocsWithField;
DirectMonotonicReader.Meta addressesMeta;
long addressesOffset;
@ -368,7 +338,7 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
@Override
public long ramBytesUsed() {
return ramBytesUsed + disiCacheFactory.ramBytesUsed();
return ramBytesUsed;
}
@Override
@ -465,18 +435,44 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
}
};
} else {
final RandomAccessInput slice = data.randomAccessSlice(entry.valuesOffset, entry.valuesLength);
if (entry.blockShift >= 0) {
// dense but split into blocks of different bits per value
final int shift = entry.blockShift;
final long mul = entry.gcd;
final int mask = (1 << shift) - 1;
return new DenseNumericDocValues(maxDoc) {
final VaryingBPVReader vBPVReader = new VaryingBPVReader(entry);
int block = -1;
long delta;
long offset;
long blockEndOffset;
LongValues values;
@Override
public long longValue() throws IOException {
return vBPVReader.getLongValue(doc);
final int block = doc >>> shift;
if (this.block != block) {
int bitsPerValue;
do {
offset = blockEndOffset;
bitsPerValue = slice.readByte(offset++);
delta = slice.readLong(offset);
offset += Long.BYTES;
if (bitsPerValue == 0) {
blockEndOffset = offset;
} else {
final int length = slice.readInt(offset);
offset += Integer.BYTES;
blockEndOffset = offset + length;
}
this.block ++;
} while (this.block != block);
values = bitsPerValue == 0 ? LongValues.ZEROES : DirectReader.getInstance(slice, bitsPerValue, offset);
}
return mul * values.get(doc & mask) + delta;
}
};
} else {
final RandomAccessInput slice = data.randomAccessSlice(entry.valuesOffset, entry.valuesLength);
final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue);
if (entry.table != null) {
final long[] table = entry.table;
@ -500,8 +496,7 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
}
} else {
// sparse
final IndexedDISI disi = disiCacheFactory.createCachedIndexedDISI(
data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numValues, entry.name);
final IndexedDISI disi = new IndexedDISI(data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numValues);
if (entry.bitsPerValue == 0) {
return new SparseNumericDocValues(disi) {
@Override
@ -510,19 +505,45 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
}
};
} else {
final RandomAccessInput slice = data.randomAccessSlice(entry.valuesOffset, entry.valuesLength);
if (entry.blockShift >= 0) {
// sparse and split into blocks of different bits per value
final int shift = entry.blockShift;
final long mul = entry.gcd;
final int mask = (1 << shift) - 1;
return new SparseNumericDocValues(disi) {
final VaryingBPVReader vBPVReader = new VaryingBPVReader(entry);
int block = -1;
long delta;
long offset;
long blockEndOffset;
LongValues values;
@Override
public long longValue() throws IOException {
final int index = disi.index();
return vBPVReader.getLongValue(index);
final int block = index >>> shift;
if (this.block != block) {
int bitsPerValue;
do {
offset = blockEndOffset;
bitsPerValue = slice.readByte(offset++);
delta = slice.readLong(offset);
offset += Long.BYTES;
if (bitsPerValue == 0) {
blockEndOffset = offset;
} else {
final int length = slice.readInt(offset);
offset += Integer.BYTES;
blockEndOffset = offset + length;
}
this.block ++;
} while (this.block != block);
values = bitsPerValue == 0 ? LongValues.ZEROES : DirectReader.getInstance(slice, bitsPerValue, offset);
}
return mul * values.get(index & mask) + delta;
}
};
} else {
final RandomAccessInput slice = data.randomAccessSlice(entry.valuesOffset, entry.valuesLength);
final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue);
if (entry.table != null) {
final long[] table = entry.table;
@ -556,20 +577,47 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
}
};
} else {
final RandomAccessInput slice = data.randomAccessSlice(entry.valuesOffset, entry.valuesLength);
if (entry.blockShift >= 0) {
final int shift = entry.blockShift;
final long mul = entry.gcd;
final long mask = (1L << shift) - 1;
return new LongValues() {
final VaryingBPVReader vBPVReader = new VaryingBPVReader(entry);
@Override
long block = -1;
long delta;
long offset;
long blockEndOffset;
LongValues values;
public long get(long index) {
try {
return vBPVReader.getLongValue(index);
} catch (IOException e) {
throw new RuntimeException(e);
final long block = index >>> shift;
if (this.block != block) {
assert block > this.block : "Reading backwards is illegal: " + this.block + " < " + block;
int bitsPerValue;
do {
offset = blockEndOffset;
try {
bitsPerValue = slice.readByte(offset++);
delta = slice.readLong(offset);
offset += Long.BYTES;
if (bitsPerValue == 0) {
blockEndOffset = offset;
} else {
final int length = slice.readInt(offset);
offset += Integer.BYTES;
blockEndOffset = offset + length;
}
} catch (IOException e) {
throw new RuntimeException(e);
}
this.block ++;
} while (this.block != block);
values = bitsPerValue == 0 ? LongValues.ZEROES : DirectReader.getInstance(slice, bitsPerValue, offset);
}
return mul * values.get(index & mask) + delta;
}
};
} else {
final RandomAccessInput slice = data.randomAccessSlice(entry.valuesOffset, entry.valuesLength);
final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue);
if (entry.table != null) {
final long[] table = entry.table;
@ -719,8 +767,7 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
}
} else {
// sparse
final IndexedDISI disi = disiCacheFactory.createCachedIndexedDISI(
data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numDocsWithField, entry.name);
final IndexedDISI disi = new IndexedDISI(data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numDocsWithField);
if (entry.minLength == entry.maxLength) {
// fixed length
final int length = entry.maxLength;
@ -821,8 +868,7 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
};
} else {
// sparse
final IndexedDISI disi = disiCacheFactory.createCachedIndexedDISI(
data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numDocsWithField, entry.name);
final IndexedDISI disi = new IndexedDISI(data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numDocsWithField);
return new BaseSortedDocValues(entry, data) {
@Override
@ -1190,8 +1236,7 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
};
} else {
// sparse
final IndexedDISI disi = disiCacheFactory.createCachedIndexedDISI(
data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numDocsWithField, entry.name);
final IndexedDISI disi = new IndexedDISI(data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numDocsWithField);
return new SortedNumericDocValues() {
boolean set;
@ -1317,8 +1362,7 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
};
} else {
// sparse
final IndexedDISI disi = disiCacheFactory.createCachedIndexedDISI(
data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numDocsWithField, entry.name);
final IndexedDISI disi = new IndexedDISI(data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numDocsWithField);
return new BaseSortedSetDocValues(entry, data) {
boolean set;
@ -1378,64 +1422,4 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
CodecUtil.checksumEntireFile(data);
}
/**
* Reader for longs split into blocks of different bits per values.
* The longs are requested by index and must be accessed in monotonically increasing order.
*/
// Note: The order requirement could be removed as the jump-tables allow for backwards iteration.
private class VaryingBPVReader {
final RandomAccessInput slice;
final NumericEntry entry;
final int shift;
final long mul;
final int mask;
long block = -1;
long delta;
long offset;
long blockEndOffset;
LongValues values;
VaryingBPVReader(NumericEntry entry) throws IOException {
this.entry = entry;
slice = data.randomAccessSlice(entry.valuesOffset, entry.valuesLength);
shift = entry.blockShift;
mul = entry.gcd;
mask = (1 << shift) - 1;
}
long getLongValue(long index) throws IOException {
final long block = index >>> shift;
if (this.block != block) {
int bitsPerValue;
do {
// If the needed block is the one directly following the current block, it is cheaper to avoid the cache
if (block != this.block+1) {
IndexedDISICacheFactory.VaryingBPVJumpTable cache;
if ((cache = disiCacheFactory.getVBPVJumpTable(entry.name, slice, entry.valuesLength)) != null) {
long candidateOffset;
if ((candidateOffset = cache.getBlockOffset(block)) != -1) {
blockEndOffset = candidateOffset;
this.block = block - 1;
}
}
}
offset = blockEndOffset;
bitsPerValue = slice.readByte(offset++);
delta = slice.readLong(offset);
offset += Long.BYTES;
if (bitsPerValue == 0) {
blockEndOffset = offset;
} else {
final int length = slice.readInt(offset);
offset += Integer.BYTES;
blockEndOffset = offset + length;
}
this.block++;
} while (this.block != block);
values = bitsPerValue == 0 ? LongValues.ZEROES : DirectReader.getInstance(slice, bitsPerValue, offset);
}
return mul * values.get(index & mask) + delta;
}
}
}

View File

@ -45,7 +45,6 @@ final class Lucene70NormsProducer extends NormsProducer implements Cloneable {
private final Map<Integer,NormsEntry> norms = new HashMap<>();
private final int maxDoc;
private IndexInput data;
private final IndexedDISICacheFactory disiCacheFactory = new IndexedDISICacheFactory();
private boolean merging;
private Map<Integer, IndexInput> disiInputs;
private Map<Integer, RandomAccessInput> dataInputs;
@ -294,9 +293,7 @@ final class Lucene70NormsProducer extends NormsProducer implements Cloneable {
} else {
// sparse
final IndexInput disiInput = getDisiInput(field, entry);
final IndexedDISI disi = disiCacheFactory.createCachedIndexedDISI(
disiInput, entry.docsWithFieldOffset + entry.docsWithFieldLength, entry.numDocsWithField, field.name);
final IndexedDISI disi = new IndexedDISI(disiInput, entry.numDocsWithField);
if (entry.bytesPerNorm == 0) {
return new SparseNormsIterator(disi) {
@Override
@ -345,12 +342,11 @@ final class Lucene70NormsProducer extends NormsProducer implements Cloneable {
@Override
public void close() throws IOException {
data.close();
disiCacheFactory.releaseAll();
}
@Override
public long ramBytesUsed() {
return 64L * norms.size() + disiCacheFactory.ramBytesUsed(); // good enough
return 64L * norms.size(); // good enough
}
@Override

View File

@ -1,302 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.util;
import java.io.IOException;
import java.util.Collection;
import org.apache.lucene.search.DocIdSetIterator;
/**
* Wrapper for OpenBitSet which creates and exposes a rank cache. The rank-cache scales to 2 billion bits.
*
* The rankCache has a long for every 2048 bits and thus has an overhead of 3.17% relative to the given bit set.
* Performance is O(1):
* 1 lookup in cache,
* a maximum of 3 sums,
* a maximum of 8 Long.bitCounts.
*
* Creation performance is equivalent to a full count of all set bits in the bit set O(n).
*
* Note: {@link #buildRankCache()} must be called once after the bit set has been created or changed and before
* calling {@link #rank(long)}.
*
* The design is based heavily on the article
* Space-Efficient, High-Performance Rank and Select Structures on Uncompressed Bit Sequences
* by Dong Zhou, David G. Andersen, Michael Kaminsky, Carnegie Mellon University, Intel Labs
* http://www.cs.cmu.edu/~dga/papers/zhou-sea2013.pdf
*/
// Note: If the total number of set bits is <= 65535, a faster rank cache would be a short for every 512 bits
// This is not available in the current implementation.
// Extending the rank beyond 2 billion bits could be done by dividing the bitmap into blocks of 2b bits and
// introducing yet another table with a rank-origo for each 2b-block
public class RankBitSet extends BitSet {
public static final int LOWER_BITS = 32; // Must be capable of addressing full Java array
public static final long LOWER_MASK = ~(~1L << (LOWER_BITS-1));
public static final int LOWER_OVER_BITS = 11;
public static final long LOWER_OVER_MASK = ~(~1L << (LOWER_OVER_BITS-1));
public static final int LOWER_OVER_SIZE = 2048; // Overflow bits handled by a lower block
public static final int BASIC_BITS = 10; // Needs to hold counts from 0-512 (513-1023 are never used)
public static final long BASIC_MASK = ~(~1L << (BASIC_BITS-1));
public static final int BASIC_OVER_BITS = 9;
public static final long BASIC_OVER_MASK = ~(~1L << (BASIC_OVER_BITS-1));
public static final int BASIC_OVER_SIZE = 512; // Overflow bits handled by a basic block
public static final int BASIC_WORDS = BASIC_OVER_SIZE /Long.SIZE; // word == long
/**
* Each entry is made up of 1 long<br/>
* Bits 63-32: 32 bit first-level absolute index.<br/>
* Bits 30+31 are unused. These could be used to signal all-set or all-unset for the block to spare a few cycles?
* Bits 29-0: 3 * 10 bit (0-1023) second-level relative index. Only numbers 0-512 are used.
*/
private long[] rankCache = null;
private final FixedBitSet inner;
private final int wlen; // Number of words (longs with FixedBitSet) in inner
public RankBitSet(int numBits) {
this(new FixedBitSet(numBits));
}
public RankBitSet(FixedBitSet inner) {
this.inner = inner;
wlen = inner.getBits().length;
}
/**
* Must be called after bits has changed and before {@link #rank} is called.
*/
public void buildRankCache() {
rankCache = new long[(length() >>> LOWER_OVER_BITS)+1];
long total = 0;
int lower = 0 ;
while (lower * LOWER_OVER_SIZE < length()) { // Full lower block processing
final int origoWordIndex = (lower * LOWER_OVER_SIZE) >>> 6;
// TODO: Some conditionals could be spared by checking once if all basic blocks are within size
final long basic1 = origoWordIndex + BASIC_WORDS <= wlen ?
BitUtil.pop_array(inner.getBits(), origoWordIndex, BASIC_WORDS) : 0;
final long basic2 = origoWordIndex + BASIC_WORDS*2 <= wlen ?
BitUtil.pop_array(inner.getBits(), origoWordIndex + BASIC_WORDS, BASIC_WORDS) : 0;
final long basic3 = origoWordIndex + BASIC_WORDS*3 <= wlen ?
BitUtil.pop_array(inner.getBits(), origoWordIndex + BASIC_WORDS *2, BASIC_WORDS) : 0;
final long basic4 = origoWordIndex + BASIC_WORDS*4 <= wlen ?
BitUtil.pop_array(inner.getBits(), origoWordIndex + BASIC_WORDS *3, BASIC_WORDS) : 0;
rankCache[lower] = total << (Long.SIZE-LOWER_BITS) |
basic1 << (BASIC_BITS *2) |
basic2 << BASIC_BITS |
basic3;
total += basic1 + basic2 + basic3 + basic4;
lower++;
}
}
/**
* Get the rank (number of set bits up to right before the index) for the given index in O(1).
* @param index offset in the originating bit set.
* @return the rank for the index.
*/
public int rank(long index) {
final long cache = rankCache[((int) (index >>> LOWER_OVER_BITS))];
// lower cache (absolute)
long rank = cache >>> (Long.SIZE-LOWER_BITS);
int startBitIndex = (int) (index & ~LOWER_OVER_MASK);
// basic blocks (relative)
if (startBitIndex < index-BASIC_OVER_SIZE) {
rank += (cache >>> (BASIC_BITS*2)) & BASIC_MASK;
startBitIndex += BASIC_OVER_SIZE;
if (startBitIndex < index-BASIC_OVER_SIZE) {
rank += (cache >>> BASIC_BITS) & BASIC_MASK;
startBitIndex += BASIC_OVER_SIZE;
if (startBitIndex < index-BASIC_OVER_SIZE) {
rank += cache & BASIC_MASK;
startBitIndex += BASIC_OVER_SIZE;
}
}
}
// long.bitcount (relative)
while(startBitIndex < index-Long.SIZE) {
rank += Long.bitCount(inner.getBits()[startBitIndex >>> 6]);
startBitIndex += Long.SIZE;
}
// Single bits (relative)
if (startBitIndex < index) {
/* System.out.println(String.format(Locale.ENGLISH,
"startBitIndex=%d, index=%d, getBits()[startBitIndex>>>6=%d]=%s, index-startBitIndex=%d, mask=%s",
startBitIndex, index, startBitIndex>>>6, Long.toBinaryString(getBits()[startBitIndex>>>6]),
index-startBitIndex, Long.toBinaryString(~(~1L << (index-startBitIndex-1)))));*/
rank += Long.bitCount(inner.getBits()[startBitIndex >>> 6] & ~(~1L << (index-startBitIndex-1)));
}
// for (int i = startBitIndex ; i < index ; i++) {
// rank += fastGet(i) ? 1 : 0;
// }
return (int) rank;
}
@Override
public long ramBytesUsed() {
return RamUsageEstimator.alignObjectSize(RamUsageEstimator.NUM_BYTES_OBJECT_REF*2 +
Integer.BYTES + Long.BYTES) +
inner.ramBytesUsed() +
(rankCache == null ? 0 :
RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + Long.BYTES*rankCache.length);
}
/* Delegations to inner bit set below */
public static FixedBitSet ensureCapacity(FixedBitSet bits, int numBits) {
return FixedBitSet.ensureCapacity(bits, numBits);
}
public static int bits2words(int numBits) {
return FixedBitSet.bits2words(numBits);
}
public static long intersectionCount(FixedBitSet a, FixedBitSet b) {
return FixedBitSet.intersectionCount(a, b);
}
public static long unionCount(FixedBitSet a, FixedBitSet b) {
return FixedBitSet.unionCount(a, b);
}
public static long andNotCount(FixedBitSet a, FixedBitSet b) {
return FixedBitSet.andNotCount(a, b);
}
@Override
public int length() {
return inner.length();
}
public long[] getBits() {
return inner.getBits();
}
@Override
public int cardinality() {
return inner.cardinality();
}
@Override
public boolean get(int index) {
return inner.get(index);
}
@Override
public void set(int index) {
inner.set(index);
}
public boolean getAndSet(int index) {
return inner.getAndSet(index);
}
@Override
public void clear(int index) {
inner.clear(index);
}
public boolean getAndClear(int index) {
return inner.getAndClear(index);
}
@Override
public int nextSetBit(int index) {
return inner.nextSetBit(index);
}
@Override
public int prevSetBit(int index) {
return inner.prevSetBit(index);
}
@Override
public void or(DocIdSetIterator iter) throws IOException {
inner.or(iter);
}
public void or(FixedBitSet other) {
inner.or(other);
}
public void xor(FixedBitSet other) {
inner.xor(other);
}
public void xor(DocIdSetIterator iter) throws IOException {
inner.xor(iter);
}
public boolean intersects(FixedBitSet other) {
return inner.intersects(other);
}
public void and(FixedBitSet other) {
inner.and(other);
}
public void andNot(FixedBitSet other) {
inner.andNot(other);
}
public boolean scanIsEmpty() {
return inner.scanIsEmpty();
}
public void flip(int startIndex, int endIndex) {
inner.flip(startIndex, endIndex);
}
public void flip(int index) {
inner.flip(index);
}
public void set(int startIndex, int endIndex) {
inner.set(startIndex, endIndex);
}
@Override
public void clear(int startIndex, int endIndex) {
inner.clear(startIndex, endIndex);
}
@Override
public int hashCode() {
return inner.hashCode();
}
public static FixedBitSet copyOf(Bits bits) {
return FixedBitSet.copyOf(bits);
}
public Bits asReadOnlyBits() {
return inner.asReadOnlyBits();
}
public static BitSet of(DocIdSetIterator it, int maxDoc) throws IOException {
return BitSet.of(it, maxDoc);
}
@Override
public int approximateCardinality() {
return inner.approximateCardinality();
}
@Override
public Collection<Accountable> getChildResources() {
return inner.getChildResources();
}
}

View File

@ -150,39 +150,6 @@ public class TestIndexedDISI extends LuceneTestCase {
}
}
}
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 testOneDocMissingFixed() throws IOException {
int maxDoc = 9699;
FixedBitSet set = new FixedBitSet(maxDoc);
set.set(0, maxDoc);
set.clear(1345);
try (Directory dir = newDirectory()) {
final int cardinality = set.cardinality();
long length;
try (IndexOutput out = dir.createOutput("foo", IOContext.DEFAULT)) {
IndexedDISI.writeBitSet(new BitSetIterator(set, cardinality), out);
length = out.getFilePointer();
}
int step = 16000;
try (IndexInput in = dir.openInput("foo", IOContext.DEFAULT)) {
IndexedDISI disi = new IndexedDISI(in, 0L, length, cardinality);
BitSetIterator disi2 = new BitSetIterator(set, cardinality);
assertAdvanceEquality(disi, disi2, step);
}
}
}
public void testRandom() throws IOException {
try (Directory dir = newDirectory()) {
@ -221,14 +188,32 @@ public class TestIndexedDISI extends LuceneTestCase {
try (IndexInput in = dir.openInput("foo", IOContext.DEFAULT)) {
IndexedDISI disi = new IndexedDISI(in, 0L, length, cardinality);
BitSetIterator disi2 = new BitSetIterator(set, cardinality);
assertSingleStepEquality(disi, disi2);
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());
}
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, cardinality);
BitSetIterator disi2 = new BitSetIterator(set, cardinality);
assertAdvanceEquality(disi, disi2, step);
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(index, disi.index());
}
}
}
@ -236,61 +221,29 @@ public class TestIndexedDISI extends LuceneTestCase {
try (IndexInput in = dir.openInput("foo", IOContext.DEFAULT)) {
IndexedDISI disi = new IndexedDISI(in, 0L, length, cardinality);
BitSetIterator disi2 = new BitSetIterator(set, cardinality);
int disi2length = set.length();
assertAdvanceExactRandomized(disi, disi2, disi2length, step);
int index = -1;
for (int target = 0; target < set.length(); ) {
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());
assertEquals(index, disi.index());
target = doc;
}
}
}
}
dir.deleteFile("foo");
}
private void assertAdvanceExactRandomized(IndexedDISI disi, BitSetIterator disi2, int disi2length, int step)
throws IOException {
int index = -1;
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());
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());
}
}
}

View File

@ -1,87 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene70;
import java.util.Locale;
import java.util.Random;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.packed.PackedInts;
public class TestLongCompressor extends LuceneTestCase {
// Simple money test that builds a collection of random longs, compresses them with LongCompressor and
// checks if all values from the compressed version are equal to the source
public void testLongCompressorMonkey() {
final int RUNS = 10;
final int[] MAX_SIZES = new int[]{0, 1, 10, 1000, 100_000};
final int[] MAX_VALUE = new int[]{0, 1, 10, 1000, 100_000};
for (int run = 0 ; run < RUNS ; run++) {
for (int maxSize: MAX_SIZES) {
int size = maxSize == 0 ? 0 : random().nextInt(maxSize);
for (int maxValue: MAX_VALUE) {
int minValue = maxValue == 0 ? 0 : random().nextInt(maxValue);
double minChance = random().nextDouble();
longCompressorMonkeyTest(run, size, minValue, maxValue, minChance, random().nextLong());
}
}
}
}
public void testVerySparse() {
final int SIZE = 674932;
final int EVERY = SIZE/896;
PackedInts.Mutable ranks = PackedInts.getMutable(674932, 16, PackedInts.DEFAULT);
for (int i = 0 ; i < SIZE; i+=EVERY) {
ranks.set(i, random().nextInt(65535));
}
PackedInts.Reader sparsed = LongCompressor.compress(ranks);
assertFalse("The input and the sparsed should not be the same", ranks == sparsed);
}
private void longCompressorMonkeyTest(
int run, int size, int minValue, int maxValue, double minChance, long randomSeed) {
final String description = String.format(Locale.ENGLISH,
"run=%d, size=%d, minValue=%d, maxValue=%d, minChance=%1.2f, seed=%d",
run, size, minValue, maxValue, minChance, randomSeed);
Random innerRandom = new Random(randomSeed);
PackedInts.Mutable expected = PackedInts.getMutable(size, PackedInts.bitsRequired(maxValue), PackedInts.DEFAULT);
for (int i = 0 ; i < size ; i++) {
if (innerRandom.nextDouble() <= minChance) {
continue;
}
expected.set(i, maxValue-minValue == 0 ? minValue : innerRandom.nextInt(maxValue-minValue)+minValue);
}
assertSparseValid(description, expected);
}
private void assertSparseValid(String description, PackedInts.Reader values) {
try {
PackedInts.Reader sparsed = LongCompressor.compress(values, values.size());
for (int i = 0; i < values.size(); i++) {
assertEquals("The value at index " + i + " should be as expected for " + description,
values.get(i), sparsed.get(i));
}
} catch (Exception e) {
throw new RuntimeException("Unexpected Exception for " + description, e);
}
}
}

View File

@ -18,13 +18,7 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.analysis.standard.StandardAnalyzer;
import org.apache.lucene.document.BinaryDocValuesField;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
@ -33,9 +27,7 @@ import org.apache.lucene.document.SortedDocValuesField;
import org.apache.lucene.document.SortedNumericDocValuesField;
import org.apache.lucene.document.SortedSetDocValuesField;
import org.apache.lucene.document.StringField;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.MMapDirectory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LuceneTestCase;
@ -131,105 +123,8 @@ public class TestDocValues extends LuceneTestCase {
iw.close();
dir.close();
}
/**
* Triggers varying bits per value codec representation for numeric.
*/
public void testNumericFieldVaryingBPV() throws Exception {
Directory dir = newDirectory();
IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig(null));
long generatedSum = 0;
for (int bpv = 2 ; bpv < 24 ; bpv+=3) {
for (int i = 0 ; i < 66000 ; i++) {
Document doc = new Document();
int max = 1 << (bpv - 1);
int value = random().nextInt(max) | max;
generatedSum += value;
//System.out.println("--- " + value);
doc.add(new NumericDocValuesField("foo", value));
iw.addDocument(doc);
}
}
iw.flush();
iw.forceMerge(1, true);
iw.commit();
DirectoryReader dr = DirectoryReader.open(iw);
LeafReader r = getOnlyLeafReader(dr);
// ok
NumericDocValues numDV = DocValues.getNumeric(r, "foo");
assertNotNull(numDV);
long sum = 0;
while (numDV.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
sum += numDV.longValue();
}
assertEquals("The sum of retrieved values should match the input", generatedSum, sum);
// assertNotNull(DocValues.getSortedNumeric(r, "foo"));
dr.close();
iw.close();
dir.close();
}
// LUCENE-8374 had a bug where a vBPV-block with BPV==0 as the very end of the numeric DocValues made it fail
public void testNumericEntryZeroesLastBlock() throws IOException {
List<Long> docValues = new ArrayList<>(2*16384);
for (int id = 0 ; id < 2*16384 ; id++) { // 2 vBPV-blocks for the dv-field
if (id < 16384) { // First vBPV-block just has semi-ramdom values
docValues.add((long) (id % 1000));
} else { // Second block is all zeroes, resulting in an extreme "1-byte for the while block"-representation
docValues.add(0L);
}
}
assertRandomAccessDV("Last block BPV=0", docValues);
}
private void assertRandomAccessDV(String designation, List<Long> docValues) throws IOException {
// Create corpus
Path zeroPath = Paths.get(System.getProperty("java.io.tmpdir"),"plain_" + random().nextInt());
Directory zeroDir = new MMapDirectory(zeroPath);
IndexWriterConfig iwc = new IndexWriterConfig(new StandardAnalyzer());
//iwc.setCodec(Codec.forName("Lucene70"));
IndexWriter iw = new IndexWriter(zeroDir, iwc);
for (int id = 0 ; id < docValues.size() ; id++) {
Document doc = new Document();
doc.add(new StringField("id", Integer.toString(id), Field.Store.YES));
doc.add(new NumericDocValuesField("dv", docValues.get(id)));
iw.addDocument(doc);
}
iw.flush();
iw.commit();
iw.forceMerge(1, true);
iw.close();
DirectoryReader dr = DirectoryReader.open(zeroDir);
for (int id = 0 ; id < docValues.size() ; id++) {
int readerIndex = dr.readerIndex(id);
// We create a new reader each time as we want to test vBPV-skipping and not sequential iteration
NumericDocValues numDV = dr.leaves().get(readerIndex).reader().getNumericDocValues("dv");
assertTrue(designation + ": There should be a value for docID " + id, numDV.advanceExact(id));
assertEquals(designation + ": The value for docID " + id + " should be as expected",
docValues.get(id), Long.valueOf(numDV.longValue()));
}
dr.close();
// Clean up
deleteAndClose(zeroDir);
Files.delete(zeroPath);
}
private void deleteAndClose(Directory dir) throws IOException {
String[] files = dir.listAll();
for (String file: files) {
dir.deleteFile(file);
}
dir.close();
}
/**
/**
* field with binary docvalues
*/
public void testBinaryField() throws Exception {

View File

@ -1,107 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.util;
import java.util.Locale;
import java.util.Random;
public class TestRankBitSet extends LuceneTestCase {
public void testSingleWord() {
RankBitSet rank = new RankBitSet(60);
rank.set(20);
rank.buildRankCache();
assertEquals("The rank at index 20 should be correct", 0, rank.rank(20));
assertEquals("The rank at index 21 should be correct", 1, rank.rank(21));
}
public void testSecondWord() {
RankBitSet rank = new RankBitSet(100);
rank.set(70);
rank.buildRankCache();
assertEquals("The rank at index 70 should be correct", 0, rank.rank(70));
assertEquals("The rank at index 71 should be correct", 1, rank.rank(71));
}
public void testThirdWord() {
RankBitSet rank = new RankBitSet(200);
rank.set(130);
rank.buildRankCache();
assertEquals("The rank at index 130 should be correct", 0, rank.rank(130));
assertEquals("The rank at index 131 should be correct", 1, rank.rank(131));
}
public void testSecondLower() {
RankBitSet rank = new RankBitSet(3000);
rank.set(2500);
rank.buildRankCache();
assertEquals("The rank at index 2500 should be correct", 0, rank.rank(2500));
assertEquals("The rank at index 2500 should be correct", 1, rank.rank(2501));
}
public void testSpecific282() {
RankBitSet rank = new RankBitSet(448);
rank.set(282);
rank.buildRankCache();
assertEquals("The rank at index 288 should be correct", 1, rank.rank(288));
}
public void testSpecific1031() {
RankBitSet rank = new RankBitSet(1446);
rank.set(1031);
rank.buildRankCache();
assertEquals("The rank at index 1057 should be correct", 1, rank.rank(1057));
}
public void testMonkeys() {
monkey(20, 8000, 40);
}
@Slow
public void testManyMonkeys() {
monkey(20, 100000, 400);
}
public void monkey(int runs, int sizeMax, int setMax) {
Random random = random();
//Random random = new Random(87);
for (int run = 0 ; run < runs ; run++) {
final int size = random.nextInt(sizeMax-1)+1;
RankBitSet rank = new RankBitSet(size);
int doSet = random.nextInt(setMax);
for (int s = 0 ; s < doSet ; s++) {
int index = random.nextInt(size);
rank.set(index);
}
rank.buildRankCache();
int setbits = 0;
for (int i = 0 ; i < size ; i++) {
assertEquals(String.format(Locale.ENGLISH, "run=%d, index=%d/%d, setbits=%d", run, i, size, setbits),
setbits, rank.rank(i));
if (rank.get(i)) {
setbits++;
}
}
}
}
}