mirror of https://github.com/apache/lucene.git
LUCENE-10315: Speed up DocIdsWriter by ForUtil (#797)
This commit is contained in:
parent
3bcc40efe9
commit
35ca2d79f7
|
@ -119,6 +119,8 @@ Optimizations
|
|||
* LUCENE-10503: Potential speedup for pure disjunctions whose clauses produce
|
||||
scores that are very close to each other. (Adrien Grand)
|
||||
|
||||
* LUCENE-10315: Use SIMD instructions to decode BKD doc IDs. (Guo Feng, Adrien Grand, Ignacio Vera)
|
||||
|
||||
Bug Fixes
|
||||
---------------------
|
||||
* LUCENE-10477: Highlighter: WeightedSpanTermExtractor.extractWeightedSpanTerms to Query#rewrite
|
||||
|
|
|
@ -80,6 +80,14 @@ final class EndiannessReverserIndexInput extends IndexInput {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readInts(int[] dst, int offset, int length) throws IOException {
|
||||
in.readInts(dst, offset, length);
|
||||
for (int i = 0; i < length; ++i) {
|
||||
dst[offset + i] = Integer.reverseBytes(dst[offset + i]);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFloats(float[] dst, int offset, int length) throws IOException {
|
||||
in.readFloats(dst, offset, length);
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.store;
|
|||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.FloatBuffer;
|
||||
import java.nio.IntBuffer;
|
||||
import java.nio.LongBuffer;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
|
@ -138,6 +139,11 @@ final class ByteBufferGuard {
|
|||
receiver.get(dst, offset, length);
|
||||
}
|
||||
|
||||
public void getInts(IntBuffer receiver, int[] dst, int offset, int length) {
|
||||
ensureValid();
|
||||
receiver.get(dst, offset, length);
|
||||
}
|
||||
|
||||
public void getFloats(FloatBuffer receiver, float[] dst, int offset, int length) {
|
||||
ensureValid();
|
||||
receiver.get(dst, offset, length);
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.nio.BufferUnderflowException;
|
|||
import java.nio.ByteBuffer;
|
||||
import java.nio.ByteOrder;
|
||||
import java.nio.FloatBuffer;
|
||||
import java.nio.IntBuffer;
|
||||
import java.nio.LongBuffer;
|
||||
|
||||
/**
|
||||
|
@ -36,6 +37,7 @@ import java.nio.LongBuffer;
|
|||
public abstract class ByteBufferIndexInput extends IndexInput implements RandomAccessInput {
|
||||
private static final FloatBuffer EMPTY_FLOATBUFFER = FloatBuffer.allocate(0);
|
||||
private static final LongBuffer EMPTY_LONGBUFFER = LongBuffer.allocate(0);
|
||||
private static final IntBuffer EMPTY_INTBUFFER = IntBuffer.allocate(0);
|
||||
|
||||
protected final long length;
|
||||
protected final long chunkSizeMask;
|
||||
|
@ -46,6 +48,7 @@ public abstract class ByteBufferIndexInput extends IndexInput implements RandomA
|
|||
protected int curBufIndex = -1;
|
||||
protected ByteBuffer curBuf; // redundant for speed: buffers[curBufIndex]
|
||||
private LongBuffer[] curLongBufferViews;
|
||||
private IntBuffer[] curIntBufferViews;
|
||||
private FloatBuffer[] curFloatBufferViews;
|
||||
|
||||
protected boolean isClone = false;
|
||||
|
@ -83,6 +86,7 @@ public abstract class ByteBufferIndexInput extends IndexInput implements RandomA
|
|||
this.curBuf = curBuf;
|
||||
curLongBufferViews = null;
|
||||
curFloatBufferViews = null;
|
||||
curIntBufferViews = null;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -176,6 +180,37 @@ public abstract class ByteBufferIndexInput extends IndexInput implements RandomA
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readInts(int[] dst, int offset, int length) throws IOException {
|
||||
// See notes about readLongs above
|
||||
if (curIntBufferViews == null) {
|
||||
curIntBufferViews = new IntBuffer[Integer.BYTES];
|
||||
for (int i = 0; i < Integer.BYTES; ++i) {
|
||||
if (i < curBuf.limit()) {
|
||||
curIntBufferViews[i] =
|
||||
curBuf.duplicate().position(i).order(ByteOrder.LITTLE_ENDIAN).asIntBuffer();
|
||||
} else {
|
||||
curIntBufferViews[i] = EMPTY_INTBUFFER;
|
||||
}
|
||||
}
|
||||
}
|
||||
try {
|
||||
final int position = curBuf.position();
|
||||
guard.getInts(
|
||||
curIntBufferViews[position & 0x03].position(position >>> 2), dst, offset, length);
|
||||
// if the above call succeeded, then we know the below sum cannot overflow
|
||||
curBuf.position(position + (length << 2));
|
||||
} catch (
|
||||
@SuppressWarnings("unused")
|
||||
BufferUnderflowException e) {
|
||||
super.readInts(dst, offset, length);
|
||||
} catch (
|
||||
@SuppressWarnings("unused")
|
||||
NullPointerException npe) {
|
||||
throw new AlreadyClosedException("Already closed: " + this);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public final void readFloats(float[] floats, int offset, int len) throws IOException {
|
||||
// See notes about readLongs above
|
||||
|
@ -503,6 +538,7 @@ public abstract class ByteBufferIndexInput extends IndexInput implements RandomA
|
|||
curBuf = null;
|
||||
curBufIndex = 0;
|
||||
curLongBufferViews = null;
|
||||
curIntBufferViews = null;
|
||||
}
|
||||
|
||||
/** Optimization of ByteBufferIndexInput for when there is only one buffer */
|
||||
|
|
|
@ -169,6 +169,20 @@ public abstract class DataInput implements Cloneable {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads a specified number of ints into an array at the specified offset.
|
||||
*
|
||||
* @param dst the array to read bytes into
|
||||
* @param offset the offset in the array to start storing ints
|
||||
* @param length the number of ints to read
|
||||
*/
|
||||
public void readInts(int[] dst, int offset, int length) throws IOException {
|
||||
Objects.checkFromIndexSize(offset, length, dst.length);
|
||||
for (int i = 0; i < length; ++i) {
|
||||
dst[offset + i] = readInt();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads a specified number of floats into an array at the specified offset.
|
||||
*
|
||||
|
|
|
@ -217,6 +217,7 @@ public class BKDReader extends PointValues {
|
|||
scratchMaxIndexPackedValue;
|
||||
private final int[] commonPrefixLengths;
|
||||
private final BKDReaderDocIDSetIterator scratchIterator;
|
||||
private final DocIdsWriter docIdsWriter;
|
||||
// if true the tree is balanced, otherwise unbalanced
|
||||
private final boolean isTreeBalanced;
|
||||
|
||||
|
@ -303,6 +304,7 @@ public class BKDReader extends PointValues {
|
|||
this.scratchDataPackedValue = scratchDataPackedValue;
|
||||
this.scratchMinIndexPackedValue = scratchMinIndexPackedValue;
|
||||
this.scratchMaxIndexPackedValue = scratchMaxIndexPackedValue;
|
||||
this.docIdsWriter = scratchIterator.docIdsWriter;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -570,7 +572,7 @@ public class BKDReader extends PointValues {
|
|||
// How many points are stored in this leaf cell:
|
||||
int count = leafNodes.readVInt();
|
||||
// No need to call grow(), it has been called up-front
|
||||
DocIdsWriter.readInts(leafNodes, count, visitor);
|
||||
docIdsWriter.readInts(leafNodes, count, visitor);
|
||||
} else {
|
||||
pushLeft();
|
||||
addAll(visitor, grown);
|
||||
|
@ -633,7 +635,7 @@ public class BKDReader extends PointValues {
|
|||
// How many points are stored in this leaf cell:
|
||||
int count = in.readVInt();
|
||||
|
||||
DocIdsWriter.readInts(in, count, iterator.docIDs);
|
||||
docIdsWriter.readInts(in, count, iterator.docIDs);
|
||||
|
||||
return count;
|
||||
}
|
||||
|
@ -1002,9 +1004,11 @@ public class BKDReader extends PointValues {
|
|||
private int offset;
|
||||
private int docID;
|
||||
final int[] docIDs;
|
||||
private final DocIdsWriter docIdsWriter;
|
||||
|
||||
public BKDReaderDocIDSetIterator(int maxPointsInLeafNode) {
|
||||
this.docIDs = new int[maxPointsInLeafNode];
|
||||
this.docIdsWriter = new DocIdsWriter(maxPointsInLeafNode);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -130,6 +130,7 @@ public class BKDWriter implements Closeable {
|
|||
private final long totalPointCount;
|
||||
|
||||
private final int maxDoc;
|
||||
private final DocIdsWriter docIdsWriter;
|
||||
|
||||
public BKDWriter(
|
||||
int maxDoc,
|
||||
|
@ -165,7 +166,7 @@ public class BKDWriter implements Closeable {
|
|||
|
||||
// Maximum number of points we hold in memory at any time
|
||||
maxPointsSortInHeap = (int) ((maxMBSortInHeap * 1024 * 1024) / (config.bytesPerDoc));
|
||||
|
||||
docIdsWriter = new DocIdsWriter(config.maxPointsInLeafNode);
|
||||
// Finally, we must be able to hold at least the leaf node in heap during build:
|
||||
if (maxPointsSortInHeap < config.maxPointsInLeafNode) {
|
||||
throw new IllegalArgumentException(
|
||||
|
@ -1288,7 +1289,7 @@ public class BKDWriter implements Closeable {
|
|||
throws IOException {
|
||||
assert count > 0 : "config.maxPointsInLeafNode=" + config.maxPointsInLeafNode;
|
||||
out.writeVInt(count);
|
||||
DocIdsWriter.writeDocIds(docIDs, start, count, out);
|
||||
docIdsWriter.writeDocIds(docIDs, start, count, out);
|
||||
}
|
||||
|
||||
private void writeLeafBlockPackedValues(
|
||||
|
|
|
@ -24,31 +24,43 @@ import org.apache.lucene.store.IndexInput;
|
|||
import org.apache.lucene.util.DocBaseBitSetIterator;
|
||||
import org.apache.lucene.util.FixedBitSet;
|
||||
|
||||
class DocIdsWriter {
|
||||
final class DocIdsWriter {
|
||||
|
||||
private DocIdsWriter() {}
|
||||
private static final byte CONTINUOUS_IDS = (byte) -2;
|
||||
private static final byte BITSET_IDS = (byte) -1;
|
||||
private static final byte DELTA_BPV_16 = (byte) 16;
|
||||
private static final byte BPV_24 = (byte) 24;
|
||||
private static final byte BPV_32 = (byte) 32;
|
||||
// These signs are legacy, should no longer be used in the writing side.
|
||||
private static final byte LEGACY_DELTA_VINT = (byte) 0;
|
||||
|
||||
static void writeDocIds(int[] docIds, int start, int count, DataOutput out) throws IOException {
|
||||
private final int[] scratch;
|
||||
|
||||
DocIdsWriter(int maxPointsInLeaf) {
|
||||
scratch = new int[maxPointsInLeaf];
|
||||
}
|
||||
|
||||
void writeDocIds(int[] docIds, int start, int count, DataOutput out) throws IOException {
|
||||
// docs can be sorted either when all docs in a block have the same value
|
||||
// or when a segment is sorted
|
||||
boolean sorted = true;
|
||||
boolean strictlySorted = true;
|
||||
int min = docIds[0];
|
||||
int max = docIds[0];
|
||||
for (int i = 1; i < count; ++i) {
|
||||
int last = docIds[start + i - 1];
|
||||
int current = docIds[start + i];
|
||||
if (last > current) {
|
||||
sorted = strictlySorted = false;
|
||||
break;
|
||||
} else if (last == current) {
|
||||
if (last >= current) {
|
||||
strictlySorted = false;
|
||||
}
|
||||
min = Math.min(min, current);
|
||||
max = Math.max(max, current);
|
||||
}
|
||||
|
||||
int min2max = docIds[start + count - 1] - docIds[start] + 1;
|
||||
int min2max = max - min + 1;
|
||||
if (strictlySorted) {
|
||||
if (min2max == count) {
|
||||
// continuous ids, typically happens when segment is sorted
|
||||
out.writeByte((byte) -2);
|
||||
out.writeByte(CONTINUOUS_IDS);
|
||||
out.writeVInt(docIds[start]);
|
||||
return;
|
||||
} else if (min2max <= (count << 4)) {
|
||||
|
@ -56,26 +68,31 @@ class DocIdsWriter {
|
|||
// Only trigger bitset optimization when max - min + 1 <= 16 * count in order to avoid
|
||||
// expanding too much storage.
|
||||
// A field with lower cardinality will have higher probability to trigger this optimization.
|
||||
out.writeByte((byte) -1);
|
||||
out.writeByte(BITSET_IDS);
|
||||
writeIdsAsBitSet(docIds, start, count, out);
|
||||
return;
|
||||
}
|
||||
}
|
||||
if (sorted) {
|
||||
out.writeByte((byte) 0);
|
||||
int previous = 0;
|
||||
for (int i = 0; i < count; ++i) {
|
||||
int doc = docIds[start + i];
|
||||
out.writeVInt(doc - previous);
|
||||
previous = doc;
|
||||
|
||||
if (min2max <= 0xFFFF) {
|
||||
out.writeByte(DELTA_BPV_16);
|
||||
for (int i = 0; i < count; i++) {
|
||||
scratch[i] = docIds[start + i] - min;
|
||||
}
|
||||
out.writeVInt(min);
|
||||
final int halfLen = count >>> 1;
|
||||
for (int i = 0; i < halfLen; ++i) {
|
||||
scratch[i] = scratch[halfLen + i] | (scratch[i] << 16);
|
||||
}
|
||||
for (int i = 0; i < halfLen; i++) {
|
||||
out.writeInt(scratch[i]);
|
||||
}
|
||||
if ((count & 1) == 1) {
|
||||
out.writeShort((short) scratch[count - 1]);
|
||||
}
|
||||
} else {
|
||||
long max = 0;
|
||||
for (int i = 0; i < count; ++i) {
|
||||
max |= Integer.toUnsignedLong(docIds[start + i]);
|
||||
}
|
||||
if (max <= 0xffffff) {
|
||||
out.writeByte((byte) 24);
|
||||
if (max <= 0xFFFFFF) {
|
||||
out.writeByte(BPV_24);
|
||||
// write them the same way we are reading them.
|
||||
int i;
|
||||
for (i = 0; i < count - 7; i += 8) {
|
||||
|
@ -103,8 +120,8 @@ class DocIdsWriter {
|
|||
out.writeByte((byte) docIds[start + i]);
|
||||
}
|
||||
} else {
|
||||
out.writeByte((byte) 32);
|
||||
for (int i = 0; i < count; ++i) {
|
||||
out.writeByte(BPV_32);
|
||||
for (int i = 0; i < count; i++) {
|
||||
out.writeInt(docIds[start + i]);
|
||||
}
|
||||
}
|
||||
|
@ -145,23 +162,26 @@ class DocIdsWriter {
|
|||
}
|
||||
|
||||
/** Read {@code count} integers into {@code docIDs}. */
|
||||
static void readInts(IndexInput in, int count, int[] docIDs) throws IOException {
|
||||
void readInts(IndexInput in, int count, int[] docIDs) throws IOException {
|
||||
final int bpv = in.readByte();
|
||||
switch (bpv) {
|
||||
case -2:
|
||||
case CONTINUOUS_IDS:
|
||||
readContinuousIds(in, count, docIDs);
|
||||
break;
|
||||
case -1:
|
||||
case BITSET_IDS:
|
||||
readBitSet(in, count, docIDs);
|
||||
break;
|
||||
case 0:
|
||||
readDeltaVInts(in, count, docIDs);
|
||||
case DELTA_BPV_16:
|
||||
readDelta16(in, count, docIDs);
|
||||
break;
|
||||
case 32:
|
||||
case BPV_24:
|
||||
readInts24(in, count, docIDs);
|
||||
break;
|
||||
case BPV_32:
|
||||
readInts32(in, count, docIDs);
|
||||
break;
|
||||
case 24:
|
||||
readInts24(in, count, docIDs);
|
||||
case LEGACY_DELTA_VINT:
|
||||
readLegacyDeltaVInts(in, count, docIDs);
|
||||
break;
|
||||
default:
|
||||
throw new IOException("Unsupported number of bits per value: " + bpv);
|
||||
|
@ -184,6 +204,15 @@ class DocIdsWriter {
|
|||
}
|
||||
}
|
||||
|
||||
private static void readLegacyDeltaVInts(IndexInput in, int count, int[] docIDs)
|
||||
throws IOException {
|
||||
int doc = 0;
|
||||
for (int i = 0; i < count; i++) {
|
||||
doc += in.readVInt();
|
||||
docIDs[i] = doc;
|
||||
}
|
||||
}
|
||||
|
||||
private static void readBitSet(IndexInput in, int count, int[] docIDs) throws IOException {
|
||||
DocIdSetIterator iterator = readBitSetIterator(in, count);
|
||||
int docId, pos = 0;
|
||||
|
@ -193,17 +222,17 @@ class DocIdsWriter {
|
|||
assert pos == count : "pos: " + pos + "count: " + count;
|
||||
}
|
||||
|
||||
private static void readDeltaVInts(IndexInput in, int count, int[] docIDs) throws IOException {
|
||||
int doc = 0;
|
||||
for (int i = 0; i < count; i++) {
|
||||
doc += in.readVInt();
|
||||
docIDs[i] = doc;
|
||||
private static void readDelta16(IndexInput in, int count, int[] docIDs) throws IOException {
|
||||
final int min = in.readVInt();
|
||||
final int halfLen = count >>> 1;
|
||||
in.readInts(docIDs, 0, halfLen);
|
||||
for (int i = 0; i < halfLen; ++i) {
|
||||
int l = docIDs[i];
|
||||
docIDs[i] = (l >>> 16) + min;
|
||||
docIDs[halfLen + i] = (l & 0xFFFF) + min;
|
||||
}
|
||||
}
|
||||
|
||||
private static void readInts32(IndexInput in, int count, int[] docIDs) throws IOException {
|
||||
for (int i = 0; i < count; i++) {
|
||||
docIDs[i] = in.readInt();
|
||||
if ((count & 1) == 1) {
|
||||
docIDs[count - 1] = Short.toUnsignedInt(in.readShort()) + min;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -227,34 +256,58 @@ class DocIdsWriter {
|
|||
}
|
||||
}
|
||||
|
||||
private static void readInts32(IndexInput in, int count, int[] docIDs) throws IOException {
|
||||
in.readInts(docIDs, 0, count);
|
||||
}
|
||||
|
||||
/**
|
||||
* Read {@code count} integers and feed the result directly to {@link
|
||||
* IntersectVisitor#visit(int)}.
|
||||
*/
|
||||
static void readInts(IndexInput in, int count, IntersectVisitor visitor) throws IOException {
|
||||
void readInts(IndexInput in, int count, IntersectVisitor visitor) throws IOException {
|
||||
final int bpv = in.readByte();
|
||||
switch (bpv) {
|
||||
case -2:
|
||||
case CONTINUOUS_IDS:
|
||||
readContinuousIds(in, count, visitor);
|
||||
break;
|
||||
case -1:
|
||||
case BITSET_IDS:
|
||||
readBitSet(in, count, visitor);
|
||||
break;
|
||||
case 0:
|
||||
readDeltaVInts(in, count, visitor);
|
||||
case DELTA_BPV_16:
|
||||
readDelta16(in, count, visitor);
|
||||
break;
|
||||
case 32:
|
||||
case BPV_24:
|
||||
readInts24(in, count, visitor);
|
||||
break;
|
||||
case BPV_32:
|
||||
readInts32(in, count, visitor);
|
||||
break;
|
||||
case 24:
|
||||
readInts24(in, count, visitor);
|
||||
case LEGACY_DELTA_VINT:
|
||||
readLegacyDeltaVInts(in, count, visitor);
|
||||
break;
|
||||
default:
|
||||
throw new IOException("Unsupported number of bits per value: " + bpv);
|
||||
}
|
||||
}
|
||||
|
||||
private static void readDeltaVInts(IndexInput in, int count, IntersectVisitor visitor)
|
||||
private static void readBitSet(IndexInput in, int count, IntersectVisitor visitor)
|
||||
throws IOException {
|
||||
DocIdSetIterator bitSetIterator = readBitSetIterator(in, count);
|
||||
visitor.visit(bitSetIterator);
|
||||
}
|
||||
|
||||
private static void readContinuousIds(IndexInput in, int count, IntersectVisitor visitor)
|
||||
throws IOException {
|
||||
int start = in.readVInt();
|
||||
int extra = start & 63;
|
||||
int offset = start - extra;
|
||||
int numBits = count + extra;
|
||||
FixedBitSet bitSet = new FixedBitSet(numBits);
|
||||
bitSet.set(extra, numBits);
|
||||
visitor.visit(new DocBaseBitSetIterator(bitSet, count, offset));
|
||||
}
|
||||
|
||||
private static void readLegacyDeltaVInts(IndexInput in, int count, IntersectVisitor visitor)
|
||||
throws IOException {
|
||||
int doc = 0;
|
||||
for (int i = 0; i < count; i++) {
|
||||
|
@ -263,10 +316,10 @@ class DocIdsWriter {
|
|||
}
|
||||
}
|
||||
|
||||
private static void readInts32(IndexInput in, int count, IntersectVisitor visitor)
|
||||
throws IOException {
|
||||
private void readDelta16(IndexInput in, int count, IntersectVisitor visitor) throws IOException {
|
||||
readDelta16(in, count, scratch);
|
||||
for (int i = 0; i < count; i++) {
|
||||
visitor.visit(in.readInt());
|
||||
visitor.visit(scratch[i]);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -291,20 +344,10 @@ class DocIdsWriter {
|
|||
}
|
||||
}
|
||||
|
||||
private static void readBitSet(IndexInput in, int count, IntersectVisitor visitor)
|
||||
throws IOException {
|
||||
DocIdSetIterator bitSetIterator = readBitSetIterator(in, count);
|
||||
visitor.visit(bitSetIterator);
|
||||
}
|
||||
|
||||
private static void readContinuousIds(IndexInput in, int count, IntersectVisitor visitor)
|
||||
throws IOException {
|
||||
int start = in.readVInt();
|
||||
int extra = start & 63;
|
||||
int offset = start - extra;
|
||||
int numBits = count + extra;
|
||||
FixedBitSet bitSet = new FixedBitSet(numBits);
|
||||
bitSet.set(extra, numBits);
|
||||
visitor.visit(new DocBaseBitSetIterator(bitSet, count, offset));
|
||||
private void readInts32(IndexInput in, int count, IntersectVisitor visitor) throws IOException {
|
||||
in.readInts(scratch, 0, count);
|
||||
for (int i = 0; i < count; i++) {
|
||||
visitor.visit(scratch[i]);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -60,6 +60,21 @@ public class TestDocIdsWriter extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
public void testCluster() throws Exception {
|
||||
int numIters = atLeast(100);
|
||||
try (Directory dir = newDirectory()) {
|
||||
for (int iter = 0; iter < numIters; ++iter) {
|
||||
int[] docIDs = new int[1 + random().nextInt(5000)];
|
||||
int min = random().nextInt(1000);
|
||||
final int bpv = TestUtil.nextInt(random(), 1, 16);
|
||||
for (int i = 0; i < docIDs.length; ++i) {
|
||||
docIDs[i] = min + TestUtil.nextInt(random(), 0, (1 << bpv) - 1);
|
||||
}
|
||||
test(dir, docIDs);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void testBitSet() throws Exception {
|
||||
int numIters = atLeast(100);
|
||||
try (Directory dir = newDirectory()) {
|
||||
|
@ -93,8 +108,9 @@ public class TestDocIdsWriter extends LuceneTestCase {
|
|||
|
||||
private void test(Directory dir, int[] ints) throws Exception {
|
||||
final long len;
|
||||
DocIdsWriter docIdsWriter = new DocIdsWriter(ints.length);
|
||||
try (IndexOutput out = dir.createOutput("tmp", IOContext.DEFAULT)) {
|
||||
DocIdsWriter.writeDocIds(ints, 0, ints.length, out);
|
||||
docIdsWriter.writeDocIds(ints, 0, ints.length, out);
|
||||
len = out.getFilePointer();
|
||||
if (random().nextBoolean()) {
|
||||
out.writeLong(0); // garbage
|
||||
|
@ -102,13 +118,13 @@ public class TestDocIdsWriter extends LuceneTestCase {
|
|||
}
|
||||
try (IndexInput in = dir.openInput("tmp", IOContext.READONCE)) {
|
||||
int[] read = new int[ints.length];
|
||||
DocIdsWriter.readInts(in, ints.length, read);
|
||||
docIdsWriter.readInts(in, ints.length, read);
|
||||
assertArrayEquals(ints, read);
|
||||
assertEquals(len, in.getFilePointer());
|
||||
}
|
||||
try (IndexInput in = dir.openInput("tmp", IOContext.READONCE)) {
|
||||
int[] read = new int[ints.length];
|
||||
DocIdsWriter.readInts(
|
||||
docIdsWriter.readInts(
|
||||
in,
|
||||
ints.length,
|
||||
new IntersectVisitor() {
|
||||
|
|
|
@ -243,6 +243,65 @@ public abstract class BaseDirectoryTestCase extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
public void testAlignedInts() throws Exception {
|
||||
try (Directory dir = getDirectory(createTempDir("testAlignedInts"))) {
|
||||
try (IndexOutput out = dir.createOutput("Ints", newIOContext(random()))) {
|
||||
out.writeInt(3);
|
||||
out.writeInt(Integer.MAX_VALUE);
|
||||
out.writeInt(-3);
|
||||
}
|
||||
try (IndexInput input = dir.openInput("Ints", newIOContext(random()))) {
|
||||
assertEquals(12, input.length());
|
||||
int[] i = new int[4];
|
||||
input.readInts(i, 1, 3);
|
||||
assertArrayEquals(new int[] {0, 3, Integer.MAX_VALUE, -3}, i);
|
||||
assertEquals(12, input.getFilePointer());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void testUnalignedInts() throws Exception {
|
||||
int padding = random().nextInt(3) + 1;
|
||||
try (Directory dir = getDirectory(createTempDir("testUnalignedInts"))) {
|
||||
try (IndexOutput out = dir.createOutput("Ints", newIOContext(random()))) {
|
||||
for (int i = 0; i < padding; i++) {
|
||||
out.writeByte((byte) 2);
|
||||
}
|
||||
out.writeInt(3);
|
||||
out.writeInt(Integer.MAX_VALUE);
|
||||
out.writeInt(-3);
|
||||
}
|
||||
try (IndexInput input = dir.openInput("Ints", newIOContext(random()))) {
|
||||
assertEquals(12 + padding, input.length());
|
||||
for (int i = 0; i < padding; i++) {
|
||||
assertEquals(2, input.readByte());
|
||||
}
|
||||
int[] i = new int[4];
|
||||
input.readInts(i, 1, 3);
|
||||
assertArrayEquals(new int[] {0, 3, Integer.MAX_VALUE, -3}, i);
|
||||
assertEquals(12 + padding, input.getFilePointer());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void testIntsUnderflow() throws Exception {
|
||||
try (Directory dir = getDirectory(createTempDir("testIntsUnderflow"))) {
|
||||
final int offset = random().nextInt(4);
|
||||
final int length = TestUtil.nextInt(random(), 1, 16);
|
||||
try (IndexOutput out = dir.createOutput("Ints", newIOContext(random()))) {
|
||||
byte[] b =
|
||||
new byte
|
||||
[offset + length * Integer.BYTES - TestUtil.nextInt(random(), 1, Integer.BYTES)];
|
||||
random().nextBytes(b);
|
||||
out.writeBytes(b, b.length);
|
||||
}
|
||||
try (IndexInput input = dir.openInput("Ints", newIOContext(random()))) {
|
||||
input.seek(offset);
|
||||
expectThrows(EOFException.class, () -> input.readInts(new int[length], 0, length));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void testAlignedFloats() throws Exception {
|
||||
try (Directory dir = getDirectory(createTempDir("testAlignedFloats"))) {
|
||||
try (IndexOutput out = dir.createOutput("Floats", newIOContext(random()))) {
|
||||
|
|
Loading…
Reference in New Issue