LUCENE-10315: Speed up DocIdsWriter by ForUtil (#797)

This commit is contained in:
gf2121 2022-04-23 19:32:02 +08:00 committed by GitHub
parent 3bcc40efe9
commit 35ca2d79f7
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 267 additions and 78 deletions

View File

@ -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

View File

@ -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);

View File

@ -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);

View File

@ -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 */

View File

@ -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.
*

View File

@ -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

View File

@ -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(

View File

@ -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]);
}
}
}

View File

@ -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() {

View File

@ -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()))) {