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. EveryindexInterval
-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.
- *Token | Clean length+ | Dirty length+ | Dirty words |
---|---|---|---|
1 byte | 0-n bytes | 0-n bytes | 0-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 Comparatori
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