LUCENE-5993: Remove DocIdBitSet, WAH8DocIdSet and PForDeltaDocIdSet.

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1629838 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Adrien Grand 2014-10-07 09:40:10 +00:00
parent 6f480ae9dd
commit 5b906eeb1b
10 changed files with 31 additions and 1600 deletions

View File

@ -32,7 +32,6 @@ import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables; import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.Bits; import org.apache.lucene.util.Bits;
import org.apache.lucene.util.RoaringDocIdSet; import org.apache.lucene.util.RoaringDocIdSet;
import org.apache.lucene.util.WAH8DocIdSet;
/** /**
* Wraps another {@link Filter}'s result and caches it. The purpose is to allow * Wraps another {@link Filter}'s result and caches it. The purpose is to allow

View File

@ -1,118 +0,0 @@
package org.apache.lucene.util;
/*
* 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.
*/
import java.util.BitSet;
import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.DocIdSetIterator;
/** Simple DocIdSet and DocIdSetIterator backed by a BitSet */
public class DocIdBitSet extends DocIdSet implements Bits {
private static final long BASE_RAM_BYTES_USED =
RamUsageEstimator.shallowSizeOfInstance(DocIdBitSet.class)
+ RamUsageEstimator.shallowSizeOfInstance(BitSet.class)
+ RamUsageEstimator.NUM_BYTES_ARRAY_HEADER; // the array that stores the bits
private final BitSet bitSet;
public DocIdBitSet(BitSet bitSet) {
this.bitSet = bitSet;
}
@Override
public DocIdSetIterator iterator() {
return new DocIdBitSetIterator(bitSet);
}
@Override
public Bits bits() {
return this;
}
/** This DocIdSet implementation is cacheable. */
@Override
public boolean isCacheable() {
return true;
}
/**
* Returns the underlying BitSet.
*/
public BitSet getBitSet() {
return this.bitSet;
}
@Override
public boolean get(int index) {
return bitSet.get(index);
}
@Override
public int length() {
// the size may not be correct...
return bitSet.size();
}
@Override
public long ramBytesUsed() {
// unfortunately this is likely underestimated if the Bitset implementation
// over-sizes the array that stores the bits
return BASE_RAM_BYTES_USED + (bitSet.size() + 7) >>> 3;
}
private static class DocIdBitSetIterator extends DocIdSetIterator {
private int docId;
private BitSet bitSet;
DocIdBitSetIterator(BitSet bitSet) {
this.bitSet = bitSet;
this.docId = -1;
}
@Override
public int docID() {
return docId;
}
@Override
public int nextDoc() {
// (docId + 1) on next line requires -1 initial value for docNr:
int d = bitSet.nextSetBit(docId + 1);
// -1 returned by BitSet.nextSetBit() when exhausted
docId = d == -1 ? NO_MORE_DOCS : d;
return docId;
}
@Override
public int advance(int target) {
int d = bitSet.nextSetBit(target);
// -1 returned by BitSet.nextSetBit() when exhausted
docId = d == -1 ? NO_MORE_DOCS : d;
return docId;
}
@Override
public long cost() {
// upper bound
return bitSet.length();
}
}
}

View File

@ -1,528 +0,0 @@
package org.apache.lucene.util;
/*
* 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.
*/
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.util.packed.PackedInts;
import org.apache.lucene.util.packed.PackedLongValues;
/**
* {@link DocIdSet} implementation based on pfor-delta encoding.
* <p>This implementation is inspired from LinkedIn's Kamikaze
* (http://data.linkedin.com/opensource/kamikaze) and Daniel Lemire's JavaFastPFOR
* (https://github.com/lemire/JavaFastPFOR).</p>
* <p>On the contrary to the original PFOR paper, exceptions are encoded with
* FOR instead of Simple16.</p>
*/
public final class PForDeltaDocIdSet extends DocIdSet implements Accountable {
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(PForDeltaDocIdSet.class);
static final int BLOCK_SIZE = 128;
static final int MAX_EXCEPTIONS = 24; // no more than 24 exceptions per block
static final PackedInts.Decoder[] DECODERS = new PackedInts.Decoder[32];
static final int[] ITERATIONS = new int[32];
static final int[] BYTE_BLOCK_COUNTS = new int[32];
static final int MAX_BYTE_BLOCK_COUNT;
static final PackedLongValues SINGLE_ZERO = PackedLongValues.packedBuilder(PackedInts.COMPACT).add(0L).build();
static final PForDeltaDocIdSet EMPTY = new PForDeltaDocIdSet(null, 0, Integer.MAX_VALUE, SINGLE_ZERO, SINGLE_ZERO);
static final int LAST_BLOCK = 1 << 5; // flag to indicate the last block
static final int HAS_EXCEPTIONS = 1 << 6;
static final int UNARY = 1 << 7;
static {
int maxByteBLockCount = 0;
for (int i = 1; i < ITERATIONS.length; ++i) {
DECODERS[i] = PackedInts.getDecoder(PackedInts.Format.PACKED, PackedInts.VERSION_CURRENT, i);
assert BLOCK_SIZE % DECODERS[i].byteValueCount() == 0;
ITERATIONS[i] = BLOCK_SIZE / DECODERS[i].byteValueCount();
BYTE_BLOCK_COUNTS[i] = ITERATIONS[i] * DECODERS[i].byteBlockCount();
maxByteBLockCount = Math.max(maxByteBLockCount, DECODERS[i].byteBlockCount());
}
MAX_BYTE_BLOCK_COUNT = maxByteBLockCount;
}
/** A builder for {@link PForDeltaDocIdSet}. */
public static class Builder {
final GrowableByteArrayDataOutput data;
final int[] buffer = new int[BLOCK_SIZE];
final int[] exceptionIndices = new int[BLOCK_SIZE];
final int[] exceptions = new int[BLOCK_SIZE];
int bufferSize;
int previousDoc;
int cardinality;
int indexInterval;
int numBlocks;
// temporary variables used when compressing blocks
final int[] freqs = new int[32];
int bitsPerValue;
int numExceptions;
int bitsPerException;
/** Sole constructor. */
public Builder() {
data = new GrowableByteArrayDataOutput(128);
bufferSize = 0;
previousDoc = -1;
indexInterval = 2;
cardinality = 0;
numBlocks = 0;
}
/** Set the index interval. Every <code>indexInterval</code>-th block will
* be stored in the index. Set to {@link Integer#MAX_VALUE} to disable indexing. */
public Builder setIndexInterval(int indexInterval) {
if (indexInterval < 1) {
throw new IllegalArgumentException("indexInterval must be >= 1");
}
this.indexInterval = indexInterval;
return this;
}
/** Add a document to this builder. Documents must be added in order. */
public Builder add(int doc) {
if (doc <= previousDoc) {
throw new IllegalArgumentException("Doc IDs must be provided in order, but previousDoc=" + previousDoc + " and doc=" + doc);
}
buffer[bufferSize++] = doc - previousDoc - 1;
if (bufferSize == BLOCK_SIZE) {
encodeBlock();
bufferSize = 0;
}
previousDoc = doc;
++cardinality;
return this;
}
/** Convenience method to add the content of a {@link DocIdSetIterator} to this builder. */
public Builder add(DocIdSetIterator it) throws IOException {
for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
add(doc);
}
return this;
}
void computeFreqs() {
Arrays.fill(freqs, 0);
for (int i = 0; i < bufferSize; ++i) {
++freqs[32 - Integer.numberOfLeadingZeros(buffer[i])];
}
}
int pforBlockSize(int bitsPerValue, int numExceptions, int bitsPerException) {
final PackedInts.Format format = PackedInts.Format.PACKED;
long blockSize = 1 // header: number of bits per value
+ format.byteCount(PackedInts.VERSION_CURRENT, BLOCK_SIZE, bitsPerValue);
if (numExceptions > 0) {
blockSize += 2 // 2 additional bytes in case of exceptions: numExceptions and bitsPerException
+ numExceptions // indices of the exceptions
+ format.byteCount(PackedInts.VERSION_CURRENT, numExceptions, bitsPerException);
}
if (bufferSize < BLOCK_SIZE) {
blockSize += 1; // length of the block
}
return (int) blockSize;
}
int unaryBlockSize() {
int deltaSum = 0;
for (int i = 0; i < BLOCK_SIZE; ++i) {
deltaSum += 1 + buffer[i];
}
int blockSize = (deltaSum + 0x07) >>> 3; // round to the next byte
++blockSize; // header
if (bufferSize < BLOCK_SIZE) {
blockSize += 1; // length of the block
}
return blockSize;
}
int computeOptimalNumberOfBits() {
computeFreqs();
bitsPerValue = 31;
numExceptions = 0;
while (bitsPerValue > 0 && freqs[bitsPerValue] == 0) {
--bitsPerValue;
}
final int actualBitsPerValue = bitsPerValue;
int blockSize = pforBlockSize(bitsPerValue, numExceptions, bitsPerException);
// Now try different values for bitsPerValue and pick the best one
for (int bitsPerValue = this.bitsPerValue - 1, numExceptions = freqs[this.bitsPerValue]; bitsPerValue >= 0 && numExceptions <= MAX_EXCEPTIONS; numExceptions += freqs[bitsPerValue--]) {
final int newBlockSize = pforBlockSize(bitsPerValue, numExceptions, actualBitsPerValue - bitsPerValue);
if (newBlockSize < blockSize) {
this.bitsPerValue = bitsPerValue;
this.numExceptions = numExceptions;
blockSize = newBlockSize;
}
}
this.bitsPerException = actualBitsPerValue - bitsPerValue;
assert bufferSize < BLOCK_SIZE || numExceptions < bufferSize;
return blockSize;
}
void pforEncode() {
if (numExceptions > 0) {
final int mask = (1 << bitsPerValue) - 1;
int ex = 0;
for (int i = 0; i < bufferSize; ++i) {
if (buffer[i] > mask) {
exceptionIndices[ex] = i;
exceptions[ex++] = buffer[i] >>> bitsPerValue;
buffer[i] &= mask;
}
}
assert ex == numExceptions;
Arrays.fill(exceptions, numExceptions, BLOCK_SIZE, 0);
}
if (bitsPerValue > 0) {
final PackedInts.Encoder encoder = PackedInts.getEncoder(PackedInts.Format.PACKED, PackedInts.VERSION_CURRENT, bitsPerValue);
final int numIterations = ITERATIONS[bitsPerValue];
encoder.encode(buffer, 0, data.bytes, data.length, numIterations);
data.length += encoder.byteBlockCount() * numIterations;
}
if (numExceptions > 0) {
assert bitsPerException > 0;
data.writeByte((byte) numExceptions);
data.writeByte((byte) bitsPerException);
final PackedInts.Encoder encoder = PackedInts.getEncoder(PackedInts.Format.PACKED, PackedInts.VERSION_CURRENT, bitsPerException);
final int numIterations = (numExceptions + encoder.byteValueCount() - 1) / encoder.byteValueCount();
encoder.encode(exceptions, 0, data.bytes, data.length, numIterations);
data.length += PackedInts.Format.PACKED.byteCount(PackedInts.VERSION_CURRENT, numExceptions, bitsPerException);
for (int i = 0; i < numExceptions; ++i) {
data.writeByte((byte) exceptionIndices[i]);
}
}
}
void unaryEncode() {
int current = 0;
for (int i = 0, doc = -1; i < BLOCK_SIZE; ++i) {
doc += 1 + buffer[i];
while (doc >= 8) {
data.writeByte((byte) current);
current = 0;
doc -= 8;
}
current |= 1 << doc;
}
if (current != 0) {
data.writeByte((byte) current);
}
}
void encodeBlock() {
final int originalLength = data.length;
Arrays.fill(buffer, bufferSize, BLOCK_SIZE, 0);
final int unaryBlockSize = unaryBlockSize();
final int pforBlockSize = computeOptimalNumberOfBits();
final int blockSize;
if (pforBlockSize <= unaryBlockSize) {
// use pfor
blockSize = pforBlockSize;
data.bytes = ArrayUtil.grow(data.bytes, data.length + blockSize + MAX_BYTE_BLOCK_COUNT);
int token = bufferSize < BLOCK_SIZE ? LAST_BLOCK : 0;
token |= bitsPerValue;
if (numExceptions > 0) {
token |= HAS_EXCEPTIONS;
}
data.writeByte((byte) token);
pforEncode();
} else {
// use unary
blockSize = unaryBlockSize;
final int token = UNARY | (bufferSize < BLOCK_SIZE ? LAST_BLOCK : 0);
data.writeByte((byte) token);
unaryEncode();
}
if (bufferSize < BLOCK_SIZE) {
data.writeByte((byte) bufferSize);
}
++numBlocks;
assert data.length - originalLength == blockSize : (data.length - originalLength) + " <> " + blockSize;
}
/** Build the {@link PForDeltaDocIdSet} instance. */
public PForDeltaDocIdSet build() {
assert bufferSize < BLOCK_SIZE;
if (cardinality == 0) {
assert previousDoc == -1;
return EMPTY;
}
encodeBlock();
final byte[] dataArr = Arrays.copyOf(data.bytes, data.length + MAX_BYTE_BLOCK_COUNT);
final int indexSize = (numBlocks - 1) / indexInterval + 1;
final PackedLongValues docIDs, offsets;
if (indexSize <= 1) {
docIDs = offsets = SINGLE_ZERO;
} else {
final int pageSize = 128;
final PackedLongValues.Builder docIDsBuilder = PackedLongValues.monotonicBuilder(pageSize, PackedInts.COMPACT);
final PackedLongValues.Builder offsetsBuilder = PackedLongValues.monotonicBuilder(pageSize, PackedInts.COMPACT);
// Now build the index
final Iterator it = new Iterator(dataArr, cardinality, Integer.MAX_VALUE, SINGLE_ZERO, SINGLE_ZERO);
index:
for (int k = 0; k < indexSize; ++k) {
docIDsBuilder.add(it.docID() + 1);
offsetsBuilder.add(it.offset);
for (int i = 0; i < indexInterval; ++i) {
it.skipBlock();
if (it.docID() == DocIdSetIterator.NO_MORE_DOCS) {
break index;
}
}
}
docIDs = docIDsBuilder.build();
offsets = offsetsBuilder.build();
}
return new PForDeltaDocIdSet(dataArr, cardinality, indexInterval, docIDs, offsets);
}
}
final byte[] data;
final PackedLongValues docIDs, offsets; // for the index
final int cardinality, indexInterval;
PForDeltaDocIdSet(byte[] data, int cardinality, int indexInterval, PackedLongValues docIDs, PackedLongValues offsets) {
this.data = data;
this.cardinality = cardinality;
this.indexInterval = indexInterval;
this.docIDs = docIDs;
this.offsets = offsets;
}
@Override
public boolean isCacheable() {
return true;
}
@Override
public DocIdSetIterator iterator() {
if (data == null) {
return null;
} else {
return new Iterator(data, cardinality, indexInterval, docIDs, offsets);
}
}
static class Iterator extends DocIdSetIterator {
// index
final int indexInterval;
final PackedLongValues docIDs, offsets;
final int cardinality;
final byte[] data;
int offset; // offset in data
final int[] nextDocs;
int i; // index in nextDeltas
final int[] nextExceptions;
int blockIdx;
int docID;
Iterator(byte[] data, int cardinality, int indexInterval, PackedLongValues docIDs, PackedLongValues offsets) {
this.data = data;
this.cardinality = cardinality;
this.indexInterval = indexInterval;
this.docIDs = docIDs;
this.offsets = offsets;
offset = 0;
nextDocs = new int[BLOCK_SIZE];
Arrays.fill(nextDocs, -1);
i = BLOCK_SIZE;
nextExceptions = new int[BLOCK_SIZE];
blockIdx = -1;
docID = -1;
}
@Override
public int docID() {
return docID;
}
void pforDecompress(byte token) {
final int bitsPerValue = token & 0x1F;
if (bitsPerValue == 0) {
Arrays.fill(nextDocs, 0);
} else {
DECODERS[bitsPerValue].decode(data, offset, nextDocs, 0, ITERATIONS[bitsPerValue]);
offset += BYTE_BLOCK_COUNTS[bitsPerValue];
}
if ((token & HAS_EXCEPTIONS) != 0) {
// there are exceptions
final int numExceptions = data[offset++];
final int bitsPerException = data[offset++];
final int numIterations = (numExceptions + DECODERS[bitsPerException].byteValueCount() - 1) / DECODERS[bitsPerException].byteValueCount();
DECODERS[bitsPerException].decode(data, offset, nextExceptions, 0, numIterations);
offset += PackedInts.Format.PACKED.byteCount(PackedInts.VERSION_CURRENT, numExceptions, bitsPerException);
for (int i = 0; i < numExceptions; ++i) {
nextDocs[data[offset++]] |= nextExceptions[i] << bitsPerValue;
}
}
for (int previousDoc = docID, i = 0; i < BLOCK_SIZE; ++i) {
final int doc = previousDoc + 1 + nextDocs[i];
previousDoc = nextDocs[i] = doc;
}
}
void unaryDecompress(byte token) {
assert (token & HAS_EXCEPTIONS) == 0;
int docID = this.docID;
for (int i = 0; i < BLOCK_SIZE; ) {
final byte b = data[offset++];
for (int bitList = BitUtil.bitList(b); bitList != 0; ++i, bitList >>>= 4) {
nextDocs[i] = docID + (bitList & 0x0F);
}
docID += 8;
}
}
void decompressBlock() {
final byte token = data[offset++];
if ((token & UNARY) != 0) {
unaryDecompress(token);
} else {
pforDecompress(token);
}
if ((token & LAST_BLOCK) != 0) {
final int blockSize = data[offset++];
Arrays.fill(nextDocs, blockSize, BLOCK_SIZE, NO_MORE_DOCS);
}
++blockIdx;
}
void skipBlock() {
assert i == BLOCK_SIZE;
decompressBlock();
docID = nextDocs[BLOCK_SIZE - 1];
}
@Override
public int nextDoc() {
if (i == BLOCK_SIZE) {
decompressBlock();
i = 0;
}
return docID = nextDocs[i++];
}
int forwardBinarySearch(int target) {
// advance forward and double the window at each step
final int indexSize = (int) docIDs.size();
int lo = Math.max(blockIdx / indexInterval, 0), hi = lo + 1;
assert blockIdx == -1 || docIDs.get(lo) <= docID;
assert lo + 1 == docIDs.size() || docIDs.get(lo + 1) > docID;
while (true) {
if (hi >= indexSize) {
hi = indexSize - 1;
break;
} else if (docIDs.get(hi) >= target) {
break;
}
final int newLo = hi;
hi += (hi - lo) << 1;
lo = newLo;
}
// we found a window containing our target, let's binary search now
while (lo <= hi) {
final int mid = (lo + hi) >>> 1;
final int midDocID = (int) docIDs.get(mid);
if (midDocID <= target) {
lo = mid + 1;
} else {
hi = mid - 1;
}
}
assert docIDs.get(hi) <= target;
assert hi + 1 == docIDs.size() || docIDs.get(hi + 1) > target;
return hi;
}
@Override
public int advance(int target) throws IOException {
assert target > docID;
if (nextDocs[BLOCK_SIZE - 1] < target) {
// not in the next block, now use the index
final int index = forwardBinarySearch(target);
final int offset = (int) offsets.get(index);
if (offset > this.offset) {
this.offset = offset;
docID = (int) docIDs.get(index) - 1;
blockIdx = index * indexInterval - 1;
while (true) {
decompressBlock();
if (nextDocs[BLOCK_SIZE - 1] >= target) {
break;
}
docID = nextDocs[BLOCK_SIZE - 1];
}
i = 0;
}
}
return slowAdvance(target);
}
@Override
public long cost() {
return cardinality;
}
}
/** Return the number of documents in this {@link DocIdSet} in constant time. */
public int cardinality() {
return cardinality;
}
@Override
public long ramBytesUsed() {
if (this == EMPTY) {
return 0L;
}
long ramBytesUsed = BASE_RAM_BYTES_USED + RamUsageEstimator.sizeOf(data);
if (docIDs != SINGLE_ZERO) {
ramBytesUsed += docIDs.ramBytesUsed();
}
if (offsets != SINGLE_ZERO) {
ramBytesUsed += offsets.ramBytesUsed();
}
return ramBytesUsed;
}
}

View File

@ -1,748 +0,0 @@
package org.apache.lucene.util;
/*
* 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.
*/
import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.Comparator;
import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.util.packed.PackedInts;
import org.apache.lucene.util.packed.PackedLongValues;
/**
* {@link DocIdSet} implementation based on word-aligned hybrid encoding on
* words of 8 bits.
* <p>This implementation doesn't support random-access but has a fast
* {@link DocIdSetIterator} which can advance in logarithmic time thanks to
* an index.</p>
* <p>The compression scheme is simplistic and should work well with sparse and
* very dense doc id sets while being only slightly larger than a
* {@link FixedBitSet} for incompressible sets (overhead&lt;2% in the worst
* case) in spite of the index.</p>
* <p><b>Format</b>: The format is byte-aligned. An 8-bits word is either clean,
* meaning composed only of zeros or ones, or dirty, meaning that it contains
* between 1 and 7 bits set. The idea is to encode sequences of clean words
* using run-length encoding and to leave sequences of dirty words as-is.</p>
* <table>
* <tr><th>Token</th><th>Clean length+</th><th>Dirty length+</th><th>Dirty words</th></tr>
* <tr><td>1 byte</td><td>0-n bytes</td><td>0-n bytes</td><td>0-n bytes</td></tr>
* </table>
* <ul>
* <li><b>Token</b> encodes whether clean means full of zeros or ones in the
* first bit, the number of clean words minus 2 on the next 3 bits and the
* number of dirty words on the last 4 bits. The higher-order bit is a
* continuation bit, meaning that the number is incomplete and needs additional
* bytes to be read.</li>
* <li><b>Clean length+</b>: If clean length has its higher-order bit set,
* you need to read a {@link DataInput#readVInt() vint}, shift it by 3 bits on
* the left side and add it to the 3 bits which have been read in the token.</li>
* <li><b>Dirty length+</b> works the same way as <b>Clean length+</b> but
* on 4 bits and for the length of dirty words.</li>
* <li><b>Dirty words</b> are the dirty words, there are <b>Dirty length</b>
* of them.</li>
* </ul>
* <p>This format cannot encode sequences of less than 2 clean words and 0 dirty
* word. The reason is that if you find a single clean word, you should rather
* encode it as a dirty word. This takes the same space as starting a new
* sequence (since you need one byte for the token) but will be lighter to
* decode. There is however an exception for the first sequence. Since the first
* sequence may start directly with a dirty word, the clean length is encoded
* directly, without subtracting 2.</p>
* <p>There is an additional restriction on the format: the sequence of dirty
* words is not allowed to contain two consecutive clean words. This restriction
* exists to make sure no space is wasted and to make sure iterators can read
* the next doc ID by reading at most 2 dirty words.</p>
* @lucene.experimental
*/
public final class WAH8DocIdSet extends DocIdSet implements Accountable {
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(WAH8DocIdSet.class);
// Minimum index interval, intervals below this value can't guarantee anymore
// that this set implementation won't be significantly larger than a FixedBitSet
// The reason is that a single sequence saves at least one byte and an index
// entry requires at most 8 bytes (2 ints) so there shouldn't be more than one
// index entry every 8 sequences
private static final int MIN_INDEX_INTERVAL = 8;
/** Default index interval. */
public static final int DEFAULT_INDEX_INTERVAL = 24;
private static final PackedLongValues SINGLE_ZERO = PackedLongValues.packedBuilder(PackedInts.COMPACT).add(0L).build();
private static WAH8DocIdSet EMPTY = new WAH8DocIdSet(new byte[0], 0, 1, SINGLE_ZERO, SINGLE_ZERO);
private static final Comparator<Iterator> SERIALIZED_LENGTH_COMPARATOR = new Comparator<Iterator>() {
@Override
public int compare(Iterator wi1, Iterator wi2) {
return wi1.in.length() - wi2.in.length();
}
};
/** Same as {@link #intersect(Collection, int)} with the default index interval. */
public static WAH8DocIdSet intersect(Collection<WAH8DocIdSet> docIdSets) {
return intersect(docIdSets, DEFAULT_INDEX_INTERVAL);
}
/**
* Compute the intersection of the provided sets. This method is much faster than
* computing the intersection manually since it operates directly at the byte level.
*/
public static WAH8DocIdSet intersect(Collection<WAH8DocIdSet> docIdSets, int indexInterval) {
switch (docIdSets.size()) {
case 0:
throw new IllegalArgumentException("There must be at least one set to intersect");
case 1:
return docIdSets.iterator().next();
}
// The logic below is similar to ConjunctionScorer
final int numSets = docIdSets.size();
final Iterator[] iterators = new Iterator[numSets];
int i = 0;
for (WAH8DocIdSet set : docIdSets) {
final Iterator it = set.iterator();
iterators[i++] = it;
}
Arrays.sort(iterators, SERIALIZED_LENGTH_COMPARATOR);
final WordBuilder builder = new WordBuilder().setIndexInterval(indexInterval);
int wordNum = 0;
main:
while (true) {
// Advance the least costly iterator first
iterators[0].advanceWord(wordNum);
wordNum = iterators[0].wordNum;
if (wordNum == DocIdSetIterator.NO_MORE_DOCS) {
break;
}
byte word = iterators[0].word;
for (i = 1; i < numSets; ++i) {
if (iterators[i].wordNum < wordNum) {
iterators[i].advanceWord(wordNum);
}
if (iterators[i].wordNum > wordNum) {
wordNum = iterators[i].wordNum;
continue main;
}
assert iterators[i].wordNum == wordNum;
word &= iterators[i].word;
if (word == 0) {
// There are common words, but they don't share any bit
++wordNum;
continue main;
}
}
// Found a common word
assert word != 0;
builder.addWord(wordNum, word);
++wordNum;
}
return builder.build();
}
/** Same as {@link #union(Collection, int)} with the default index interval. */
public static WAH8DocIdSet union(Collection<WAH8DocIdSet> docIdSets) {
return union(docIdSets, DEFAULT_INDEX_INTERVAL);
}
/**
* Compute the union of the provided sets. This method is much faster than
* computing the union manually since it operates directly at the byte level.
*/
public static WAH8DocIdSet union(Collection<WAH8DocIdSet> docIdSets, int indexInterval) {
switch (docIdSets.size()) {
case 0:
return EMPTY;
case 1:
return docIdSets.iterator().next();
}
// The logic below is very similar to DisjunctionScorer
final int numSets = docIdSets.size();
final PriorityQueue<Iterator> iterators = new PriorityQueue<WAH8DocIdSet.Iterator>(numSets) {
@Override
protected boolean lessThan(Iterator a, Iterator b) {
return a.wordNum < b.wordNum;
}
};
for (WAH8DocIdSet set : docIdSets) {
Iterator iterator = set.iterator();
iterator.nextWord();
iterators.add(iterator);
}
Iterator top = iterators.top();
if (top.wordNum == Integer.MAX_VALUE) {
return EMPTY;
}
int wordNum = top.wordNum;
byte word = top.word;
final WordBuilder builder = new WordBuilder().setIndexInterval(indexInterval);
while (true) {
top.nextWord();
iterators.updateTop();
top = iterators.top();
if (top.wordNum == wordNum) {
word |= top.word;
} else {
builder.addWord(wordNum, word);
if (top.wordNum == Integer.MAX_VALUE) {
break;
}
wordNum = top.wordNum;
word = top.word;
}
}
return builder.build();
}
static int wordNum(int docID) {
assert docID >= 0;
return docID >>> 3;
}
/** Word-based builder. */
static class WordBuilder {
final GrowableByteArrayDataOutput out;
final GrowableByteArrayDataOutput dirtyWords;
int clean;
int lastWordNum;
int numSequences;
int indexInterval;
int cardinality;
boolean reverse;
WordBuilder() {
out = new GrowableByteArrayDataOutput(1024);
dirtyWords = new GrowableByteArrayDataOutput(128);
clean = 0;
lastWordNum = -1;
numSequences = 0;
indexInterval = DEFAULT_INDEX_INTERVAL;
cardinality = 0;
}
/** Set the index interval. Smaller index intervals improve performance of
* {@link DocIdSetIterator#advance(int)} but make the {@link DocIdSet}
* larger. An index interval <code>i</code> makes the index add an overhead
* which is at most <code>4/i</code>, but likely much less.The default index
* interval is <code>8</code>, meaning the index has an overhead of at most
* 50%. To disable indexing, you can pass {@link Integer#MAX_VALUE} as an
* index interval. */
public WordBuilder setIndexInterval(int indexInterval) {
if (indexInterval < MIN_INDEX_INTERVAL) {
throw new IllegalArgumentException("indexInterval must be >= " + MIN_INDEX_INTERVAL);
}
this.indexInterval = indexInterval;
return this;
}
void writeHeader(boolean reverse, int cleanLength, int dirtyLength) throws IOException {
final int cleanLengthMinus2 = cleanLength - 2;
assert cleanLengthMinus2 >= 0;
assert dirtyLength >= 0;
int token = ((cleanLengthMinus2 & 0x03) << 4) | (dirtyLength & 0x07);
if (reverse) {
token |= 1 << 7;
}
if (cleanLengthMinus2 > 0x03) {
token |= 1 << 6;
}
if (dirtyLength > 0x07) {
token |= 1 << 3;
}
out.writeByte((byte) token);
if (cleanLengthMinus2 > 0x03) {
out.writeVInt(cleanLengthMinus2 >>> 2);
}
if (dirtyLength > 0x07) {
out.writeVInt(dirtyLength >>> 3);
}
}
private boolean sequenceIsConsistent() {
for (int i = 1; i < dirtyWords.length; ++i) {
assert dirtyWords.bytes[i-1] != 0 || dirtyWords.bytes[i] != 0;
assert dirtyWords.bytes[i-1] != (byte) 0xFF || dirtyWords.bytes[i] != (byte) 0xFF;
}
return true;
}
void writeSequence() {
assert sequenceIsConsistent();
try {
writeHeader(reverse, clean, dirtyWords.length);
} catch (IOException cannotHappen) {
throw new AssertionError(cannotHappen);
}
out.writeBytes(dirtyWords.bytes, 0, dirtyWords.length);
dirtyWords.length = 0;
++numSequences;
}
void addWord(int wordNum, byte word) {
assert wordNum > lastWordNum;
assert word != 0;
if (!reverse) {
if (lastWordNum == -1) {
clean = 2 + wordNum; // special case for the 1st sequence
dirtyWords.writeByte(word);
} else {
switch (wordNum - lastWordNum) {
case 1:
if (word == (byte) 0xFF && dirtyWords.bytes[dirtyWords.length-1] == (byte) 0xFF) {
--dirtyWords.length;
writeSequence();
reverse = true;
clean = 2;
} else {
dirtyWords.writeByte(word);
}
break;
case 2:
dirtyWords.writeByte((byte) 0);
dirtyWords.writeByte(word);
break;
default:
writeSequence();
clean = wordNum - lastWordNum - 1;
dirtyWords.writeByte(word);
}
}
} else {
assert lastWordNum >= 0;
switch (wordNum - lastWordNum) {
case 1:
if (word == (byte) 0xFF) {
if (dirtyWords.length == 0) {
++clean;
} else if (dirtyWords.bytes[dirtyWords.length - 1] == (byte) 0xFF) {
--dirtyWords.length;
writeSequence();
clean = 2;
} else {
dirtyWords.writeByte(word);
}
} else {
dirtyWords.writeByte(word);
}
break;
case 2:
dirtyWords.writeByte((byte) 0);
dirtyWords.writeByte(word);
break;
default:
writeSequence();
reverse = false;
clean = wordNum - lastWordNum - 1;
dirtyWords.writeByte(word);
}
}
lastWordNum = wordNum;
cardinality += BitUtil.bitCount(word);
}
/** Build a new {@link WAH8DocIdSet}. */
public WAH8DocIdSet build() {
if (cardinality == 0) {
assert lastWordNum == -1;
return EMPTY;
}
writeSequence();
final byte[] data = Arrays.copyOf(out.bytes, out.length);
// Now build the index
final int valueCount = (numSequences - 1) / indexInterval + 1;
final PackedLongValues indexPositions, indexWordNums;
if (valueCount <= 1) {
indexPositions = indexWordNums = SINGLE_ZERO;
} else {
final int pageSize = 128;
final PackedLongValues.Builder positions = PackedLongValues.monotonicBuilder(pageSize, PackedInts.COMPACT);
final PackedLongValues.Builder wordNums = PackedLongValues.monotonicBuilder(pageSize, PackedInts.COMPACT);
positions.add(0L);
wordNums.add(0L);
final Iterator it = new Iterator(data, cardinality, Integer.MAX_VALUE, SINGLE_ZERO, SINGLE_ZERO);
assert it.in.getPosition() == 0;
assert it.wordNum == -1;
for (int i = 1; i < valueCount; ++i) {
// skip indexInterval sequences
for (int j = 0; j < indexInterval; ++j) {
final boolean readSequence = it.readSequence();
assert readSequence;
it.skipDirtyBytes();
}
final int position = it.in.getPosition();
final int wordNum = it.wordNum;
positions.add(position);
wordNums.add(wordNum + 1);
}
indexPositions = positions.build();
indexWordNums = wordNums.build();
}
return new WAH8DocIdSet(data, cardinality, indexInterval, indexPositions, indexWordNums);
}
}
/** A builder for {@link WAH8DocIdSet}s. */
public static final class Builder extends WordBuilder {
private int lastDocID;
private int wordNum, word;
/** Sole constructor */
public Builder() {
super();
lastDocID = -1;
wordNum = -1;
word = 0;
}
/** Add a document to this builder. Documents must be added in order. */
public Builder add(int docID) {
if (docID <= lastDocID) {
throw new IllegalArgumentException("Doc ids must be added in-order, got " + docID + " which is <= lastDocID=" + lastDocID);
}
final int wordNum = wordNum(docID);
if (this.wordNum == -1) {
this.wordNum = wordNum;
word = 1 << (docID & 0x07);
} else if (wordNum == this.wordNum) {
word |= 1 << (docID & 0x07);
} else {
addWord(this.wordNum, (byte) word);
this.wordNum = wordNum;
word = 1 << (docID & 0x07);
}
lastDocID = docID;
return this;
}
/** Add the content of the provided {@link DocIdSetIterator}. */
public Builder add(DocIdSetIterator disi) throws IOException {
for (int doc = disi.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = disi.nextDoc()) {
add(doc);
}
return this;
}
@Override
public Builder setIndexInterval(int indexInterval) {
return (Builder) super.setIndexInterval(indexInterval);
}
@Override
public WAH8DocIdSet build() {
if (this.wordNum != -1) {
addWord(wordNum, (byte) word);
}
return super.build();
}
}
// where the doc IDs are stored
private final byte[] data;
private final int cardinality;
private final int indexInterval;
// index for advance(int)
private final PackedLongValues positions, wordNums; // wordNums[i] starts at the sequence at positions[i]
WAH8DocIdSet(byte[] data, int cardinality, int indexInterval, PackedLongValues positions, PackedLongValues wordNums) {
this.data = data;
this.cardinality = cardinality;
this.indexInterval = indexInterval;
this.positions = positions;
this.wordNums = wordNums;
}
@Override
public boolean isCacheable() {
return true;
}
@Override
public Iterator iterator() {
return new Iterator(data, cardinality, indexInterval, positions, wordNums);
}
static int readCleanLength(ByteArrayDataInput in, int token) {
int len = (token >>> 4) & 0x07;
final int startPosition = in.getPosition();
if ((len & 0x04) != 0) {
len = (len & 0x03) | (in.readVInt() << 2);
}
if (startPosition != 1) {
len += 2;
}
return len;
}
static int readDirtyLength(ByteArrayDataInput in, int token) {
int len = token & 0x0F;
if ((len & 0x08) != 0) {
len = (len & 0x07) | (in.readVInt() << 3);
}
return len;
}
static class Iterator extends DocIdSetIterator {
/* Using the index can be costly for close targets. */
static int indexThreshold(int cardinality, int indexInterval) {
// Short sequences encode for 3 words (2 clean words and 1 dirty byte),
// don't advance if we are going to read less than 3 x indexInterval
// sequences
long indexThreshold = 3L * 3 * indexInterval;
return (int) Math.min(Integer.MAX_VALUE, indexThreshold);
}
final ByteArrayDataInput in;
final int cardinality;
final int indexInterval;
final PackedLongValues positions, wordNums;
final int indexThreshold;
int allOnesLength;
int dirtyLength;
int wordNum; // byte offset
byte word; // current word
int bitList; // list of bits set in the current word
int sequenceNum; // in which sequence are we?
int docID;
Iterator(byte[] data, int cardinality, int indexInterval, PackedLongValues positions, PackedLongValues wordNums) {
this.in = new ByteArrayDataInput(data);
this.cardinality = cardinality;
this.indexInterval = indexInterval;
this.positions = positions;
this.wordNums = wordNums;
wordNum = -1;
word = 0;
bitList = 0;
sequenceNum = -1;
docID = -1;
indexThreshold = indexThreshold(cardinality, indexInterval);
}
boolean readSequence() {
if (in.eof()) {
wordNum = Integer.MAX_VALUE;
return false;
}
final int token = in.readByte() & 0xFF;
if ((token & (1 << 7)) == 0) {
final int cleanLength = readCleanLength(in, token);
wordNum += cleanLength;
} else {
allOnesLength = readCleanLength(in, token);
}
dirtyLength = readDirtyLength(in, token);
assert in.length() - in.getPosition() >= dirtyLength : in.getPosition() + " " + in.length() + " " + dirtyLength;
++sequenceNum;
return true;
}
void skipDirtyBytes(int count) {
assert count >= 0;
assert count <= allOnesLength + dirtyLength;
wordNum += count;
if (count <= allOnesLength) {
allOnesLength -= count;
} else {
count -= allOnesLength;
allOnesLength = 0;
in.skipBytes(count);
dirtyLength -= count;
}
}
void skipDirtyBytes() {
wordNum += allOnesLength + dirtyLength;
in.skipBytes(dirtyLength);
allOnesLength = 0;
dirtyLength = 0;
}
void nextWord() {
if (allOnesLength > 0) {
word = (byte) 0xFF;
++wordNum;
--allOnesLength;
return;
}
if (dirtyLength > 0) {
word = in.readByte();
++wordNum;
--dirtyLength;
if (word != 0) {
return;
}
if (dirtyLength > 0) {
word = in.readByte();
++wordNum;
--dirtyLength;
assert word != 0; // never more than one consecutive 0
return;
}
}
if (readSequence()) {
nextWord();
}
}
int forwardBinarySearch(int targetWordNum) {
// advance forward and double the window at each step
final int indexSize = (int) wordNums.size();
int lo = sequenceNum / indexInterval, hi = lo + 1;
assert sequenceNum == -1 || wordNums.get(lo) <= wordNum;
assert lo + 1 == wordNums.size() || wordNums.get(lo + 1) > wordNum;
while (true) {
if (hi >= indexSize) {
hi = indexSize - 1;
break;
} else if (wordNums.get(hi) >= targetWordNum) {
break;
}
final int newLo = hi;
hi += (hi - lo) << 1;
lo = newLo;
}
// we found a window containing our target, let's binary search now
while (lo <= hi) {
final int mid = (lo + hi) >>> 1;
final int midWordNum = (int) wordNums.get(mid);
if (midWordNum <= targetWordNum) {
lo = mid + 1;
} else {
hi = mid - 1;
}
}
assert wordNums.get(hi) <= targetWordNum;
assert hi+1 == wordNums.size() || wordNums.get(hi + 1) > targetWordNum;
return hi;
}
void advanceWord(int targetWordNum) {
assert targetWordNum > wordNum;
int delta = targetWordNum - wordNum;
if (delta <= allOnesLength + dirtyLength + 1) {
skipDirtyBytes(delta - 1);
} else {
skipDirtyBytes();
assert dirtyLength == 0;
if (delta > indexThreshold) {
// use the index
final int i = forwardBinarySearch(targetWordNum);
final int position = (int) positions.get(i);
if (position > in.getPosition()) { // if the binary search returned a backward offset, don't move
wordNum = (int) wordNums.get(i) - 1;
in.setPosition(position);
sequenceNum = i * indexInterval - 1;
}
}
while (true) {
if (!readSequence()) {
return;
}
delta = targetWordNum - wordNum;
if (delta <= allOnesLength + dirtyLength + 1) {
if (delta > 1) {
skipDirtyBytes(delta - 1);
}
break;
}
skipDirtyBytes();
}
}
nextWord();
}
@Override
public int docID() {
return docID;
}
@Override
public int nextDoc() throws IOException {
if (bitList != 0) { // there are remaining bits in the current word
docID = (wordNum << 3) | ((bitList & 0x0F) - 1);
bitList >>>= 4;
return docID;
}
nextWord();
if (wordNum == Integer.MAX_VALUE) {
return docID = NO_MORE_DOCS;
}
bitList = BitUtil.bitList(word);
assert bitList != 0;
docID = (wordNum << 3) | ((bitList & 0x0F) - 1);
bitList >>>= 4;
return docID;
}
@Override
public int advance(int target) throws IOException {
assert target > docID;
final int targetWordNum = wordNum(target);
if (targetWordNum > wordNum) {
advanceWord(targetWordNum);
bitList = BitUtil.bitList(word);
}
return slowAdvance(target);
}
@Override
public long cost() {
return cardinality;
}
}
/** Return the number of documents in this {@link DocIdSet} in constant time. */
public int cardinality() {
return cardinality;
}
@Override
public long ramBytesUsed() {
if (this == EMPTY) {
return 0L;
}
long ramBytesUsed = BASE_RAM_BYTES_USED + RamUsageEstimator.sizeOf(data);
if (positions != SINGLE_ZERO) {
ramBytesUsed += positions.ramBytesUsed();
}
if (wordNums != SINGLE_ZERO) {
ramBytesUsed += wordNums.ramBytesUsed();
}
return ramBytesUsed;
}
}

View File

@ -34,7 +34,7 @@ import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.search.FilteredQuery.FilterStrategy; import org.apache.lucene.search.FilteredQuery.FilterStrategy;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Bits; import org.apache.lucene.util.Bits;
import org.apache.lucene.util.DocIdBitSet; import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil; import org.apache.lucene.util.TestUtil;
@ -101,10 +101,10 @@ public class TestFilteredQuery extends LuceneTestCase {
@Override @Override
public DocIdSet getDocIdSet (LeafReaderContext context, Bits acceptDocs) { public DocIdSet getDocIdSet (LeafReaderContext context, Bits acceptDocs) {
if (acceptDocs == null) acceptDocs = new Bits.MatchAllBits(5); if (acceptDocs == null) acceptDocs = new Bits.MatchAllBits(5);
BitSet bitset = new BitSet(5); FixedBitSet bitset = new FixedBitSet(context.reader().maxDoc());
if (acceptDocs.get(1)) bitset.set(1); if (acceptDocs.get(1)) bitset.set(1);
if (acceptDocs.get(3)) bitset.set(3); if (acceptDocs.get(3)) bitset.set(3);
return new DocIdBitSet(bitset); return bitset;
} }
}; };
} }
@ -183,9 +183,9 @@ public class TestFilteredQuery extends LuceneTestCase {
@Override @Override
public DocIdSet getDocIdSet (LeafReaderContext context, Bits acceptDocs) { public DocIdSet getDocIdSet (LeafReaderContext context, Bits acceptDocs) {
assertNull("acceptDocs should be null, as we have an index without deletions", acceptDocs); assertNull("acceptDocs should be null, as we have an index without deletions", acceptDocs);
BitSet bitset = new BitSet(5); FixedBitSet bitset = new FixedBitSet(context.reader().maxDoc());
bitset.set(0, 5); bitset.set(0, Math.min(5, bitset.length()));
return new DocIdBitSet(bitset); return bitset;
} }
}; };
} }

View File

@ -1,22 +1,21 @@
package org.apache.lucene.search; package org.apache.lucene.search;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.DocIdBitSet;
import org.apache.lucene.util.LuceneTestCase;
import java.util.BitSet;
import java.io.IOException; import java.io.IOException;
import java.util.BitSet;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.IndexWriterConfig.OpenMode; import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.Term;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.analysis.MockAnalyzer; import org.apache.lucene.util.Bits;
import org.apache.lucene.document.Document; import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.LuceneTestCase;
/* /*
* Licensed to the Apache Software Foundation (ASF) under one or more * Licensed to the Apache Software Foundation (ASF) under one or more
@ -38,7 +37,7 @@ import org.apache.lucene.document.Document;
public class TestScorerPerf extends LuceneTestCase { public class TestScorerPerf extends LuceneTestCase {
boolean validate = true; // set to false when doing performance testing boolean validate = true; // set to false when doing performance testing
BitSet[] sets; FixedBitSet[] sets;
Term[] terms; Term[] terms;
IndexSearcher s; IndexSearcher s;
IndexReader r; IndexReader r;
@ -81,16 +80,16 @@ public class TestScorerPerf extends LuceneTestCase {
} }
public BitSet randBitSet(int sz, int numBitsToSet) { public FixedBitSet randBitSet(int sz, int numBitsToSet) {
BitSet set = new BitSet(sz); FixedBitSet set = new FixedBitSet(sz);
for (int i=0; i<numBitsToSet; i++) { for (int i=0; i<numBitsToSet; i++) {
set.set(random().nextInt(sz)); set.set(random().nextInt(sz));
} }
return set; return set;
} }
public BitSet[] randBitSets(int numSets, int setSize) { public FixedBitSet[] randBitSets(int numSets, int setSize) {
BitSet[] sets = new BitSet[numSets]; FixedBitSet[] sets = new FixedBitSet[numSets];
for (int i=0; i<sets.length; i++) { for (int i=0; i<sets.length; i++) {
sets[i] = randBitSet(setSize, random().nextInt(setSize)); sets[i] = randBitSet(setSize, random().nextInt(setSize));
} }
@ -123,9 +122,9 @@ public class TestScorerPerf extends LuceneTestCase {
public static class MatchingHitCollector extends CountingHitCollector { public static class MatchingHitCollector extends CountingHitCollector {
BitSet answer; FixedBitSet answer;
int pos=-1; int pos=-1;
public MatchingHitCollector(BitSet answer) { public MatchingHitCollector(FixedBitSet answer) {
this.answer = answer; this.answer = answer;
} }
@ -140,18 +139,18 @@ public class TestScorerPerf extends LuceneTestCase {
} }
BitSet addClause(BooleanQuery bq, BitSet result) { FixedBitSet addClause(BooleanQuery bq, FixedBitSet result) {
final BitSet rnd = sets[random().nextInt(sets.length)]; final FixedBitSet rnd = sets[random().nextInt(sets.length)];
Query q = new ConstantScoreQuery(new Filter() { Query q = new ConstantScoreQuery(new Filter() {
@Override @Override
public DocIdSet getDocIdSet (LeafReaderContext context, Bits acceptDocs) { public DocIdSet getDocIdSet (LeafReaderContext context, Bits acceptDocs) {
assertNull("acceptDocs should be null, as we have an index without deletions", acceptDocs); assertNull("acceptDocs should be null, as we have an index without deletions", acceptDocs);
return new DocIdBitSet(rnd); return rnd;
} }
}); });
bq.add(q, BooleanClause.Occur.MUST); bq.add(q, BooleanClause.Occur.MUST);
if (validate) { if (validate) {
if (result==null) result = (BitSet)rnd.clone(); if (result==null) result = rnd.clone();
else result.and(rnd); else result.and(rnd);
} }
return result; return result;
@ -164,7 +163,7 @@ public class TestScorerPerf extends LuceneTestCase {
for (int i=0; i<iter; i++) { for (int i=0; i<iter; i++) {
int nClauses = random().nextInt(maxClauses-1)+2; // min 2 clauses int nClauses = random().nextInt(maxClauses-1)+2; // min 2 clauses
BooleanQuery bq = new BooleanQuery(); BooleanQuery bq = new BooleanQuery();
BitSet result=null; FixedBitSet result=null;
for (int j=0; j<nClauses; j++) { for (int j=0; j<nClauses; j++) {
result = addClause(bq,result); result = addClause(bq,result);
} }
@ -188,7 +187,7 @@ public class TestScorerPerf extends LuceneTestCase {
for (int i=0; i<iter; i++) { for (int i=0; i<iter; i++) {
int oClauses = random().nextInt(maxOuterClauses-1)+2; int oClauses = random().nextInt(maxOuterClauses-1)+2;
BooleanQuery oq = new BooleanQuery(); BooleanQuery oq = new BooleanQuery();
BitSet result=null; FixedBitSet result=null;
for (int o=0; o<oClauses; o++) { for (int o=0; o<oClauses; o++) {

View File

@ -1,38 +0,0 @@
package org.apache.lucene.util;
import java.io.IOException;
import java.util.BitSet;
import org.junit.Ignore;
/*
* 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.
*/
public class TestDocIdBitSet extends BaseDocIdSetTestCase<DocIdBitSet> {
@Override
public DocIdBitSet copyOf(BitSet bs, int length) throws IOException {
return new DocIdBitSet((BitSet) bs.clone());
}
@Override
@Ignore("no access to the internals of this impl")
public void testRamBytesUsed() throws IOException {
super.testRamBytesUsed();
}
}

View File

@ -52,7 +52,7 @@ public class TestDocIdSetBuilder extends LuceneTestCase {
final FixedBitSet ref = new FixedBitSet(maxDoc); final FixedBitSet ref = new FixedBitSet(maxDoc);
for (int i = 0; i < numIterators; ++i) { for (int i = 0; i < numIterators; ++i) {
final int baseInc = 200000 + random().nextInt(10000); final int baseInc = 200000 + random().nextInt(10000);
WAH8DocIdSet.Builder b = new WAH8DocIdSet.Builder(); RoaringDocIdSet.Builder b = new RoaringDocIdSet.Builder(maxDoc);
for (int doc = random().nextInt(100); doc < maxDoc; doc += baseInc + random().nextInt(10000)) { for (int doc = random().nextInt(100); doc < maxDoc; doc += baseInc + random().nextInt(10000)) {
b.add(doc); b.add(doc);
ref.set(doc); ref.set(doc);
@ -73,11 +73,11 @@ public class TestDocIdSetBuilder extends LuceneTestCase {
// try upgrades // try upgrades
final int doc = random().nextInt(maxDoc); final int doc = random().nextInt(maxDoc);
ref.set(doc); ref.set(doc);
builder.or(new WAH8DocIdSet.Builder().add(doc).build().iterator()); builder.or(new RoaringDocIdSet.Builder(maxDoc).add(doc).build().iterator());
} }
for (int i = 0; i < numIterators; ++i) { for (int i = 0; i < numIterators; ++i) {
final int baseInc = 2 + random().nextInt(10000); final int baseInc = 2 + random().nextInt(10000);
WAH8DocIdSet.Builder b = new WAH8DocIdSet.Builder(); RoaringDocIdSet.Builder b = new RoaringDocIdSet.Builder(maxDoc);
for (int doc = random().nextInt(10000); doc < maxDoc; doc += baseInc + random().nextInt(2000)) { for (int doc = random().nextInt(10000); doc < maxDoc; doc += baseInc + random().nextInt(2000)) {
b.add(doc); b.add(doc);
ref.set(doc); ref.set(doc);

View File

@ -1,41 +0,0 @@
package org.apache.lucene.util;
/*
* 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.
*/
import java.io.IOException;
import java.util.BitSet;
public class TestPForDeltaDocIdSet extends BaseDocIdSetTestCase<PForDeltaDocIdSet> {
@Override
public PForDeltaDocIdSet copyOf(BitSet bs, int length) throws IOException {
final PForDeltaDocIdSet.Builder builder = new PForDeltaDocIdSet.Builder().setIndexInterval(TestUtil.nextInt(random(), 1, 20));
for (int doc = bs.nextSetBit(0); doc != -1; doc = bs.nextSetBit(doc + 1)) {
builder.add(doc);
}
return builder.build();
}
@Override
public void assertEquals(int numBits, BitSet ds1, PForDeltaDocIdSet ds2)
throws IOException {
super.assertEquals(numBits, ds1, ds2);
assertEquals(ds1.cardinality(), ds2.cardinality());
}
}

View File

@ -1,94 +0,0 @@
package org.apache.lucene.util;
/*
* 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.
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.BitSet;
import java.util.List;
public class TestWAH8DocIdSet extends BaseDocIdSetTestCase<WAH8DocIdSet> {
@Override
public WAH8DocIdSet copyOf(BitSet bs, int length) throws IOException {
final int indexInterval = TestUtil.nextInt(random(), 8, 256);
final WAH8DocIdSet.Builder builder = new WAH8DocIdSet.Builder().setIndexInterval(indexInterval);
for (int i = bs.nextSetBit(0); i != -1; i = bs.nextSetBit(i + 1)) {
builder.add(i);
}
return builder.build();
}
@Override
public void assertEquals(int numBits, BitSet ds1, WAH8DocIdSet ds2)
throws IOException {
super.assertEquals(numBits, ds1, ds2);
assertEquals(ds1.cardinality(), ds2.cardinality());
}
public void testUnion() throws IOException {
final int numBits = TestUtil.nextInt(random(), 100, 1 << 20);
final int numDocIdSets = TestUtil.nextInt(random(), 0, 4);
final List<BitSet> fixedSets = new ArrayList<>(numDocIdSets);
for (int i = 0; i < numDocIdSets; ++i) {
fixedSets.add(randomSet(numBits, random().nextFloat() / 16));
}
final List<WAH8DocIdSet> compressedSets = new ArrayList<>(numDocIdSets);
for (BitSet set : fixedSets) {
compressedSets.add(copyOf(set, numBits));
}
final WAH8DocIdSet union = WAH8DocIdSet.union(compressedSets);
final BitSet expected = new BitSet(numBits);
for (BitSet set : fixedSets) {
for (int doc = set.nextSetBit(0); doc != -1; doc = set.nextSetBit(doc + 1)) {
expected.set(doc);
}
}
assertEquals(numBits, expected, union);
}
public void testIntersection() throws IOException {
final int numBits = TestUtil.nextInt(random(), 100, 1 << 20);
final int numDocIdSets = TestUtil.nextInt(random(), 1, 4);
final List<BitSet> fixedSets = new ArrayList<>(numDocIdSets);
for (int i = 0; i < numDocIdSets; ++i) {
fixedSets.add(randomSet(numBits, random().nextFloat()));
}
final List<WAH8DocIdSet> compressedSets = new ArrayList<>(numDocIdSets);
for (BitSet set : fixedSets) {
compressedSets.add(copyOf(set, numBits));
}
final WAH8DocIdSet union = WAH8DocIdSet.intersect(compressedSets);
final BitSet expected = new BitSet(numBits);
expected.set(0, expected.size());
for (BitSet set : fixedSets) {
for (int previousDoc = -1, doc = set.nextSetBit(0); ; previousDoc = doc, doc = set.nextSetBit(doc + 1)) {
if (doc == -1) {
expected.clear(previousDoc + 1, set.size());
break;
} else {
expected.clear(previousDoc + 1, doc);
}
}
}
assertEquals(numBits, expected, union);
}
}