diff --git a/lucene/core/src/java/org/apache/lucene/search/CachingWrapperFilter.java b/lucene/core/src/java/org/apache/lucene/search/CachingWrapperFilter.java index 9126b2274b5..3ec2975db17 100644 --- a/lucene/core/src/java/org/apache/lucene/search/CachingWrapperFilter.java +++ b/lucene/core/src/java/org/apache/lucene/search/CachingWrapperFilter.java @@ -32,7 +32,6 @@ import org.apache.lucene.util.Accountable; import org.apache.lucene.util.Accountables; import org.apache.lucene.util.Bits; 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 diff --git a/lucene/core/src/java/org/apache/lucene/util/DocIdBitSet.java b/lucene/core/src/java/org/apache/lucene/util/DocIdBitSet.java deleted file mode 100644 index 39db438d2a5..00000000000 --- a/lucene/core/src/java/org/apache/lucene/util/DocIdBitSet.java +++ /dev/null @@ -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(); - } - } -} diff --git a/lucene/core/src/java/org/apache/lucene/util/PForDeltaDocIdSet.java b/lucene/core/src/java/org/apache/lucene/util/PForDeltaDocIdSet.java deleted file mode 100644 index a4d690de7e6..00000000000 --- a/lucene/core/src/java/org/apache/lucene/util/PForDeltaDocIdSet.java +++ /dev/null @@ -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. - *

This implementation is inspired from LinkedIn's Kamikaze - * (http://data.linkedin.com/opensource/kamikaze) and Daniel Lemire's JavaFastPFOR - * (https://github.com/lemire/JavaFastPFOR).

- *

On the contrary to the original PFOR paper, exceptions are encoded with - * FOR instead of Simple16.

- */ -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 indexInterval-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; - } - -} diff --git a/lucene/core/src/java/org/apache/lucene/util/WAH8DocIdSet.java b/lucene/core/src/java/org/apache/lucene/util/WAH8DocIdSet.java deleted file mode 100644 index 51191b863fe..00000000000 --- a/lucene/core/src/java/org/apache/lucene/util/WAH8DocIdSet.java +++ /dev/null @@ -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. - *

This implementation doesn't support random-access but has a fast - * {@link DocIdSetIterator} which can advance in logarithmic time thanks to - * an index.

- *

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<2% in the worst - * case) in spite of the index.

- *

Format: 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.

- * - * - * - *
TokenClean length+Dirty length+Dirty words
1 byte0-n bytes0-n bytes0-n bytes
- * - *

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.

- *

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.

- * @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 SERIALIZED_LENGTH_COMPARATOR = new Comparator() { - @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 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 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 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 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 iterators = new PriorityQueue(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 i makes the index add an overhead - * which is at most 4/i, but likely much less.The default index - * interval is 8, 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; - } - -} diff --git a/lucene/core/src/test/org/apache/lucene/search/TestFilteredQuery.java b/lucene/core/src/test/org/apache/lucene/search/TestFilteredQuery.java index 914cb2227bd..323ffb6d5fd 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestFilteredQuery.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestFilteredQuery.java @@ -34,7 +34,7 @@ import org.apache.lucene.search.BooleanClause.Occur; import org.apache.lucene.search.FilteredQuery.FilterStrategy; import org.apache.lucene.store.Directory; 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.LuceneTestCase; import org.apache.lucene.util.TestUtil; @@ -101,10 +101,10 @@ public class TestFilteredQuery extends LuceneTestCase { @Override public DocIdSet getDocIdSet (LeafReaderContext context, Bits acceptDocs) { 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(3)) bitset.set(3); - return new DocIdBitSet(bitset); + return bitset; } }; } @@ -183,9 +183,9 @@ public class TestFilteredQuery extends LuceneTestCase { @Override public DocIdSet getDocIdSet (LeafReaderContext context, Bits acceptDocs) { assertNull("acceptDocs should be null, as we have an index without deletions", acceptDocs); - BitSet bitset = new BitSet(5); - bitset.set(0, 5); - return new DocIdBitSet(bitset); + FixedBitSet bitset = new FixedBitSet(context.reader().maxDoc()); + bitset.set(0, Math.min(5, bitset.length())); + return bitset; } }; } diff --git a/lucene/core/src/test/org/apache/lucene/search/TestScorerPerf.java b/lucene/core/src/test/org/apache/lucene/search/TestScorerPerf.java index b5a5a94897d..6405173e6fe 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestScorerPerf.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestScorerPerf.java @@ -1,22 +1,21 @@ 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.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.IndexReader; import org.apache.lucene.index.IndexWriter; -import org.apache.lucene.index.Term; 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.analysis.MockAnalyzer; -import org.apache.lucene.document.Document; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.FixedBitSet; +import org.apache.lucene.util.LuceneTestCase; /* * 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 { boolean validate = true; // set to false when doing performance testing - BitSet[] sets; + FixedBitSet[] sets; Term[] terms; IndexSearcher s; IndexReader r; @@ -81,16 +80,16 @@ public class TestScorerPerf extends LuceneTestCase { } - public BitSet randBitSet(int sz, int numBitsToSet) { - BitSet set = new BitSet(sz); + public FixedBitSet randBitSet(int sz, int numBitsToSet) { + FixedBitSet set = new FixedBitSet(sz); for (int i=0; i { - - @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(); - } - -} diff --git a/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java b/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java index d180cea9e6c..9ea581ef70d 100644 --- a/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java +++ b/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java @@ -52,7 +52,7 @@ public class TestDocIdSetBuilder extends LuceneTestCase { final FixedBitSet ref = new FixedBitSet(maxDoc); for (int i = 0; i < numIterators; ++i) { 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)) { b.add(doc); ref.set(doc); @@ -73,11 +73,11 @@ public class TestDocIdSetBuilder extends LuceneTestCase { // try upgrades final int doc = random().nextInt(maxDoc); 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) { 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)) { b.add(doc); ref.set(doc); diff --git a/lucene/core/src/test/org/apache/lucene/util/TestPForDeltaDocIdSet.java b/lucene/core/src/test/org/apache/lucene/util/TestPForDeltaDocIdSet.java deleted file mode 100644 index 0439730de4e..00000000000 --- a/lucene/core/src/test/org/apache/lucene/util/TestPForDeltaDocIdSet.java +++ /dev/null @@ -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 { - - @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()); - } - -} diff --git a/lucene/core/src/test/org/apache/lucene/util/TestWAH8DocIdSet.java b/lucene/core/src/test/org/apache/lucene/util/TestWAH8DocIdSet.java deleted file mode 100644 index 1689d020b98..00000000000 --- a/lucene/core/src/test/org/apache/lucene/util/TestWAH8DocIdSet.java +++ /dev/null @@ -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 { - - @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 fixedSets = new ArrayList<>(numDocIdSets); - for (int i = 0; i < numDocIdSets; ++i) { - fixedSets.add(randomSet(numBits, random().nextFloat() / 16)); - } - final List 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 fixedSets = new ArrayList<>(numDocIdSets); - for (int i = 0; i < numDocIdSets; ++i) { - fixedSets.add(randomSet(numBits, random().nextFloat())); - } - final List 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); - } - -}