mirror of https://github.com/apache/lucene.git
LUCENE-10417: Revert "LUCENE-10315" (#706)
This commit is contained in:
parent
d9c2e46824
commit
b0ca227862
|
@ -221,8 +221,6 @@ Optimizations
|
|||
|
||||
* LUCENE-10388: Remove MultiLevelSkipListReader#SkipBuffer to make JVM less confused. (Guo Feng)
|
||||
|
||||
* LUCENE-10315: Use SIMD instructions to decode BKD doc IDs. (Guo Feng, Adrien Grand, Ignacio Vera)
|
||||
|
||||
* LUCENE-10367: Optimize CoveringQuery for the case when the minimum number of
|
||||
matching clauses is a constant. (LuYunCheng via Adrien Grand)
|
||||
|
||||
|
|
|
@ -80,14 +80,6 @@ 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,7 +19,6 @@ 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;
|
||||
|
||||
|
@ -139,11 +138,6 @@ 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,7 +22,6 @@ import java.nio.BufferUnderflowException;
|
|||
import java.nio.ByteBuffer;
|
||||
import java.nio.ByteOrder;
|
||||
import java.nio.FloatBuffer;
|
||||
import java.nio.IntBuffer;
|
||||
import java.nio.LongBuffer;
|
||||
|
||||
/**
|
||||
|
@ -37,7 +36,6 @@ 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;
|
||||
|
@ -48,7 +46,6 @@ 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;
|
||||
|
@ -86,7 +83,6 @@ public abstract class ByteBufferIndexInput extends IndexInput implements RandomA
|
|||
this.curBuf = curBuf;
|
||||
curLongBufferViews = null;
|
||||
curFloatBufferViews = null;
|
||||
curIntBufferViews = null;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -180,37 +176,6 @@ 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
|
||||
|
@ -538,7 +503,6 @@ 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,20 +169,6 @@ 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.
|
||||
*
|
||||
|
|
|
@ -1,108 +0,0 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.util.bkd;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.lucene.store.DataInput;
|
||||
import org.apache.lucene.store.DataOutput;
|
||||
|
||||
final class BKDForUtil {
|
||||
|
||||
private final int[] tmp;
|
||||
|
||||
BKDForUtil(int maxPointsInLeaf) {
|
||||
// For encode16/decode16, we do not need to use tmp array.
|
||||
// For encode24/decode24, we need a (3/4 * maxPointsInLeaf) length tmp array.
|
||||
// For encode32/decode32, we reuse the scratch in DocIdsWriter.
|
||||
// So (3/4 * maxPointsInLeaf) is enough here.
|
||||
final int len = (maxPointsInLeaf >>> 2) * 3;
|
||||
tmp = new int[len];
|
||||
}
|
||||
|
||||
void encode16(int len, int[] ints, DataOutput out) throws IOException {
|
||||
final int halfLen = len >>> 1;
|
||||
for (int i = 0; i < halfLen; ++i) {
|
||||
ints[i] = ints[halfLen + i] | (ints[i] << 16);
|
||||
}
|
||||
for (int i = 0; i < halfLen; i++) {
|
||||
out.writeInt(ints[i]);
|
||||
}
|
||||
if ((len & 1) == 1) {
|
||||
out.writeShort((short) ints[len - 1]);
|
||||
}
|
||||
}
|
||||
|
||||
void encode32(int off, int len, int[] ints, DataOutput out) throws IOException {
|
||||
for (int i = 0; i < len; i++) {
|
||||
out.writeInt(ints[off + i]);
|
||||
}
|
||||
}
|
||||
|
||||
void encode24(int off, int len, int[] ints, DataOutput out) throws IOException {
|
||||
final int quarterLen = len >>> 2;
|
||||
final int quarterLen3 = quarterLen * 3;
|
||||
for (int i = 0; i < quarterLen3; ++i) {
|
||||
tmp[i] = ints[off + i] << 8;
|
||||
}
|
||||
for (int i = 0; i < quarterLen; i++) {
|
||||
final int longIdx = off + i + quarterLen3;
|
||||
tmp[i] |= ints[longIdx] >>> 16;
|
||||
tmp[i + quarterLen] |= (ints[longIdx] >>> 8) & 0xFF;
|
||||
tmp[i + quarterLen * 2] |= ints[longIdx] & 0xFF;
|
||||
}
|
||||
for (int i = 0; i < quarterLen3; ++i) {
|
||||
out.writeInt(tmp[i]);
|
||||
}
|
||||
|
||||
final int remainder = len & 0x3;
|
||||
for (int i = 0; i < remainder; i++) {
|
||||
out.writeInt(ints[quarterLen * 4 + i]);
|
||||
}
|
||||
}
|
||||
|
||||
void decode16(DataInput in, int[] ints, int len, final int base) throws IOException {
|
||||
final int halfLen = len >>> 1;
|
||||
in.readInts(ints, 0, halfLen);
|
||||
for (int i = 0; i < halfLen; ++i) {
|
||||
int l = ints[i];
|
||||
ints[i] = (l >>> 16) + base;
|
||||
ints[halfLen + i] = (l & 0xFFFF) + base;
|
||||
}
|
||||
if ((len & 1) == 1) {
|
||||
ints[len - 1] = Short.toUnsignedInt(in.readShort()) + base;
|
||||
}
|
||||
}
|
||||
|
||||
void decode24(DataInput in, int[] ints, int len) throws IOException {
|
||||
final int quarterLen = len >>> 2;
|
||||
final int quarterLen3 = quarterLen * 3;
|
||||
in.readInts(tmp, 0, quarterLen3);
|
||||
for (int i = 0; i < quarterLen3; ++i) {
|
||||
ints[i] = tmp[i] >>> 8;
|
||||
}
|
||||
for (int i = 0; i < quarterLen; i++) {
|
||||
ints[i + quarterLen3] =
|
||||
((tmp[i] & 0xFF) << 16)
|
||||
| ((tmp[i + quarterLen] & 0xFF) << 8)
|
||||
| (tmp[i + quarterLen * 2] & 0xFF);
|
||||
}
|
||||
int remainder = len & 0x3;
|
||||
if (remainder > 0) {
|
||||
in.readInts(ints, quarterLen << 2, remainder);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -217,7 +217,6 @@ 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;
|
||||
|
||||
|
@ -304,7 +303,6 @@ public class BKDReader extends PointValues {
|
|||
this.scratchDataPackedValue = scratchDataPackedValue;
|
||||
this.scratchMinIndexPackedValue = scratchMinIndexPackedValue;
|
||||
this.scratchMaxIndexPackedValue = scratchMaxIndexPackedValue;
|
||||
this.docIdsWriter = scratchIterator.docIdsWriter;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -572,7 +570,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);
|
||||
|
@ -635,7 +633,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;
|
||||
}
|
||||
|
@ -1004,11 +1002,9 @@ 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,7 +130,6 @@ public class BKDWriter implements Closeable {
|
|||
private final long totalPointCount;
|
||||
|
||||
private final int maxDoc;
|
||||
private final DocIdsWriter docIdsWriter;
|
||||
|
||||
public BKDWriter(
|
||||
int maxDoc,
|
||||
|
@ -166,7 +165,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(
|
||||
|
@ -1289,7 +1288,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,46 +24,31 @@ import org.apache.lucene.store.IndexInput;
|
|||
import org.apache.lucene.util.DocBaseBitSetIterator;
|
||||
import org.apache.lucene.util.FixedBitSet;
|
||||
|
||||
final class DocIdsWriter {
|
||||
class DocIdsWriter {
|
||||
|
||||
private static final byte CONTINUOUS_IDS = (byte) -2;
|
||||
private static final byte BITSET_IDS = (byte) -1;
|
||||
private static final byte DELTA_FOR_UTIL = (byte) 32 + 16;
|
||||
private static final byte BPV_24 = (byte) 32 + 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;
|
||||
private static final byte LEGACY_BPV_24 = (byte) 24;
|
||||
private DocIdsWriter() {}
|
||||
|
||||
private final BKDForUtil forUtil;
|
||||
private final int[] scratch;
|
||||
|
||||
DocIdsWriter(int maxPointsInLeaf) {
|
||||
scratch = new int[maxPointsInLeaf];
|
||||
forUtil = new BKDForUtil(maxPointsInLeaf);
|
||||
}
|
||||
|
||||
void writeDocIds(int[] docIds, int start, int count, DataOutput out) throws IOException {
|
||||
static 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) {
|
||||
if (last > current) {
|
||||
sorted = strictlySorted = false;
|
||||
break;
|
||||
} else if (last == current) {
|
||||
strictlySorted = false;
|
||||
}
|
||||
min = Math.min(min, current);
|
||||
max = Math.max(max, current);
|
||||
}
|
||||
|
||||
int min2max = max - min + 1;
|
||||
int min2max = docIds[start + count - 1] - docIds[start] + 1;
|
||||
if (strictlySorted) {
|
||||
if (min2max == count) {
|
||||
// continuous ids, typically happens when segment is sorted
|
||||
out.writeByte(CONTINUOUS_IDS);
|
||||
out.writeByte((byte) -2);
|
||||
out.writeVInt(docIds[start]);
|
||||
return;
|
||||
} else if (min2max <= (count << 4)) {
|
||||
|
@ -71,26 +56,57 @@ final 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(BITSET_IDS);
|
||||
out.writeByte((byte) -1);
|
||||
writeIdsAsBitSet(docIds, start, count, out);
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
if (min2max <= 0xFFFF) {
|
||||
out.writeByte(DELTA_FOR_UTIL);
|
||||
for (int i = 0; i < count; i++) {
|
||||
scratch[i] = docIds[start + i] - min;
|
||||
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;
|
||||
}
|
||||
out.writeVInt(min);
|
||||
forUtil.encode16(count, scratch, out);
|
||||
} else {
|
||||
if (max <= 0xFFFFFF) {
|
||||
out.writeByte(BPV_24);
|
||||
forUtil.encode24(start, count, docIds, out);
|
||||
long max = 0;
|
||||
for (int i = 0; i < count; ++i) {
|
||||
max |= Integer.toUnsignedLong(docIds[start + i]);
|
||||
}
|
||||
if (max <= 0xffffff) {
|
||||
out.writeByte((byte) 24);
|
||||
// write them the same way we are reading them.
|
||||
int i;
|
||||
for (i = 0; i < count - 7; i += 8) {
|
||||
int doc1 = docIds[start + i];
|
||||
int doc2 = docIds[start + i + 1];
|
||||
int doc3 = docIds[start + i + 2];
|
||||
int doc4 = docIds[start + i + 3];
|
||||
int doc5 = docIds[start + i + 4];
|
||||
int doc6 = docIds[start + i + 5];
|
||||
int doc7 = docIds[start + i + 6];
|
||||
int doc8 = docIds[start + i + 7];
|
||||
long l1 = (doc1 & 0xffffffL) << 40 | (doc2 & 0xffffffL) << 16 | ((doc3 >>> 8) & 0xffffL);
|
||||
long l2 =
|
||||
(doc3 & 0xffL) << 56
|
||||
| (doc4 & 0xffffffL) << 32
|
||||
| (doc5 & 0xffffffL) << 8
|
||||
| ((doc6 >> 16) & 0xffL);
|
||||
long l3 = (doc6 & 0xffffL) << 48 | (doc7 & 0xffffffL) << 24 | (doc8 & 0xffffffL);
|
||||
out.writeLong(l1);
|
||||
out.writeLong(l2);
|
||||
out.writeLong(l3);
|
||||
}
|
||||
for (; i < count; ++i) {
|
||||
out.writeShort((short) (docIds[start + i] >>> 8));
|
||||
out.writeByte((byte) docIds[start + i]);
|
||||
}
|
||||
} else {
|
||||
out.writeByte(BPV_32);
|
||||
forUtil.encode32(start, count, docIds, out);
|
||||
out.writeByte((byte) 32);
|
||||
for (int i = 0; i < count; ++i) {
|
||||
out.writeInt(docIds[start + i]);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -129,48 +145,29 @@ final class DocIdsWriter {
|
|||
}
|
||||
|
||||
/** Read {@code count} integers into {@code docIDs}. */
|
||||
void readInts(IndexInput in, int count, int[] docIDs) throws IOException {
|
||||
static void readInts(IndexInput in, int count, int[] docIDs) throws IOException {
|
||||
final int bpv = in.readByte();
|
||||
switch (bpv) {
|
||||
case CONTINUOUS_IDS:
|
||||
case -2:
|
||||
readContinuousIds(in, count, docIDs);
|
||||
break;
|
||||
case BITSET_IDS:
|
||||
case -1:
|
||||
readBitSet(in, count, docIDs);
|
||||
break;
|
||||
case DELTA_FOR_UTIL:
|
||||
readDelta16(in, count, docIDs);
|
||||
case 0:
|
||||
readDeltaVInts(in, count, docIDs);
|
||||
break;
|
||||
case BPV_24:
|
||||
readInts24(in, count, docIDs);
|
||||
break;
|
||||
case BPV_32:
|
||||
case 32:
|
||||
readInts32(in, count, docIDs);
|
||||
break;
|
||||
case LEGACY_DELTA_VINT:
|
||||
readLegacyDeltaVInts(in, count, docIDs);
|
||||
break;
|
||||
case LEGACY_BPV_24:
|
||||
readLegacyInts24(in, count, docIDs);
|
||||
case 24:
|
||||
readInts24(in, count, docIDs);
|
||||
break;
|
||||
default:
|
||||
throw new IOException("Unsupported number of bits per value: " + bpv);
|
||||
}
|
||||
}
|
||||
|
||||
private void readDelta16(IndexInput in, int count, int[] docIDs) throws IOException {
|
||||
final int min = in.readVInt();
|
||||
forUtil.decode16(in, docIDs, count, min);
|
||||
}
|
||||
|
||||
private void readInts24(IndexInput in, int count, int[] docIDs) throws IOException {
|
||||
forUtil.decode24(in, docIDs, count);
|
||||
}
|
||||
|
||||
private void readInts32(IndexInput in, int count, int[] docIDs) throws IOException {
|
||||
in.readInts(docIDs, 0, count);
|
||||
}
|
||||
|
||||
private static DocIdSetIterator readBitSetIterator(IndexInput in, int count) throws IOException {
|
||||
int offsetWords = in.readVInt();
|
||||
int longLen = in.readVInt();
|
||||
|
@ -196,8 +193,7 @@ final class DocIdsWriter {
|
|||
assert pos == count : "pos: " + pos + "count: " + count;
|
||||
}
|
||||
|
||||
private static void readLegacyDeltaVInts(IndexInput in, int count, int[] docIDs)
|
||||
throws IOException {
|
||||
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();
|
||||
|
@ -205,7 +201,13 @@ final class DocIdsWriter {
|
|||
}
|
||||
}
|
||||
|
||||
private static void readLegacyInts24(IndexInput in, int count, int[] docIDs) throws IOException {
|
||||
private static void readInts32(IndexInput in, int count, int[] docIDs) throws IOException {
|
||||
for (int i = 0; i < count; i++) {
|
||||
docIDs[i] = in.readInt();
|
||||
}
|
||||
}
|
||||
|
||||
private static void readInts24(IndexInput in, int count, int[] docIDs) throws IOException {
|
||||
int i;
|
||||
for (i = 0; i < count - 7; i += 8) {
|
||||
long l1 = in.readLong();
|
||||
|
@ -229,36 +231,30 @@ final class DocIdsWriter {
|
|||
* Read {@code count} integers and feed the result directly to {@link
|
||||
* IntersectVisitor#visit(int)}.
|
||||
*/
|
||||
void readInts(IndexInput in, int count, IntersectVisitor visitor) throws IOException {
|
||||
static void readInts(IndexInput in, int count, IntersectVisitor visitor) throws IOException {
|
||||
final int bpv = in.readByte();
|
||||
switch (bpv) {
|
||||
case CONTINUOUS_IDS:
|
||||
case -2:
|
||||
readContinuousIds(in, count, visitor);
|
||||
break;
|
||||
case BITSET_IDS:
|
||||
case -1:
|
||||
readBitSet(in, count, visitor);
|
||||
break;
|
||||
case DELTA_FOR_UTIL:
|
||||
readDelta16(in, count, visitor);
|
||||
case 0:
|
||||
readDeltaVInts(in, count, visitor);
|
||||
break;
|
||||
case BPV_24:
|
||||
readInts24(in, count, visitor);
|
||||
break;
|
||||
case BPV_32:
|
||||
case 32:
|
||||
readInts32(in, count, visitor);
|
||||
break;
|
||||
case LEGACY_DELTA_VINT:
|
||||
readLegacyDeltaVInts(in, count, visitor);
|
||||
break;
|
||||
case LEGACY_BPV_24:
|
||||
readLegacyInts24(in, count, visitor);
|
||||
case 24:
|
||||
readInts24(in, count, visitor);
|
||||
break;
|
||||
default:
|
||||
throw new IOException("Unsupported number of bits per value: " + bpv);
|
||||
}
|
||||
}
|
||||
|
||||
private static void readLegacyDeltaVInts(IndexInput in, int count, IntersectVisitor visitor)
|
||||
private static void readDeltaVInts(IndexInput in, int count, IntersectVisitor visitor)
|
||||
throws IOException {
|
||||
int doc = 0;
|
||||
for (int i = 0; i < count; i++) {
|
||||
|
@ -267,7 +263,14 @@ final class DocIdsWriter {
|
|||
}
|
||||
}
|
||||
|
||||
private static void readLegacyInts24(IndexInput in, int count, IntersectVisitor visitor)
|
||||
private static void readInts32(IndexInput in, int count, IntersectVisitor visitor)
|
||||
throws IOException {
|
||||
for (int i = 0; i < count; i++) {
|
||||
visitor.visit(in.readInt());
|
||||
}
|
||||
}
|
||||
|
||||
private static void readInts24(IndexInput in, int count, IntersectVisitor visitor)
|
||||
throws IOException {
|
||||
int i;
|
||||
for (i = 0; i < count - 7; i += 8) {
|
||||
|
@ -304,26 +307,4 @@ final class DocIdsWriter {
|
|||
bitSet.set(extra, numBits);
|
||||
visitor.visit(new DocBaseBitSetIterator(bitSet, count, offset));
|
||||
}
|
||||
|
||||
private void readDelta16(IndexInput in, int count, IntersectVisitor visitor) throws IOException {
|
||||
final int min = in.readVInt();
|
||||
forUtil.decode16(in, scratch, count, min);
|
||||
for (int i = 0; i < count; i++) {
|
||||
visitor.visit(scratch[i]);
|
||||
}
|
||||
}
|
||||
|
||||
private void readInts24(IndexInput in, int count, IntersectVisitor visitor) throws IOException {
|
||||
forUtil.decode24(in, scratch, count);
|
||||
for (int i = 0; i < count; i++) {
|
||||
visitor.visit(scratch[i]);
|
||||
}
|
||||
}
|
||||
|
||||
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]);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,7 +22,6 @@ import java.util.HashSet;
|
|||
import java.util.Set;
|
||||
import org.apache.lucene.index.PointValues.IntersectVisitor;
|
||||
import org.apache.lucene.index.PointValues.Relation;
|
||||
import org.apache.lucene.store.DataOutput;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
|
@ -33,14 +32,6 @@ import org.apache.lucene.tests.util.TestUtil;
|
|||
public class TestDocIdsWriter extends LuceneTestCase {
|
||||
|
||||
public void testRandom() throws Exception {
|
||||
innerTestRandom(false);
|
||||
}
|
||||
|
||||
public void testLegacyRandom() throws Exception {
|
||||
innerTestRandom(true);
|
||||
}
|
||||
|
||||
private void innerTestRandom(boolean legacy) throws Exception {
|
||||
int numIters = atLeast(100);
|
||||
try (Directory dir = newDirectory()) {
|
||||
for (int iter = 0; iter < numIters; ++iter) {
|
||||
|
@ -49,20 +40,12 @@ public class TestDocIdsWriter extends LuceneTestCase {
|
|||
for (int i = 0; i < docIDs.length; ++i) {
|
||||
docIDs[i] = TestUtil.nextInt(random(), 0, (1 << bpv) - 1);
|
||||
}
|
||||
test(dir, docIDs, legacy);
|
||||
test(dir, docIDs);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void testSorted() throws Exception {
|
||||
innerTestSorted(false);
|
||||
}
|
||||
|
||||
public void testLegacySorted() throws Exception {
|
||||
innerTestSorted(true);
|
||||
}
|
||||
|
||||
private void innerTestSorted(boolean legacy) throws Exception {
|
||||
int numIters = atLeast(100);
|
||||
try (Directory dir = newDirectory()) {
|
||||
for (int iter = 0; iter < numIters; ++iter) {
|
||||
|
@ -72,22 +55,7 @@ public class TestDocIdsWriter extends LuceneTestCase {
|
|||
docIDs[i] = TestUtil.nextInt(random(), 0, (1 << bpv) - 1);
|
||||
}
|
||||
Arrays.sort(docIDs);
|
||||
test(dir, docIDs, legacy);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
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, false);
|
||||
test(dir, docIDs);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -103,7 +71,7 @@ public class TestDocIdsWriter extends LuceneTestCase {
|
|||
set.add(small + random().nextInt(size * 16));
|
||||
}
|
||||
int[] docIDs = set.stream().mapToInt(t -> t).sorted().toArray();
|
||||
test(dir, docIDs, false);
|
||||
test(dir, docIDs);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -118,20 +86,15 @@ public class TestDocIdsWriter extends LuceneTestCase {
|
|||
for (int i = 0; i < docIDs.length; i++) {
|
||||
docIDs[i] = start + i;
|
||||
}
|
||||
test(dir, docIDs, false);
|
||||
test(dir, docIDs);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void test(Directory dir, int[] ints, boolean legacy) throws Exception {
|
||||
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)) {
|
||||
if (legacy) {
|
||||
legacyWriteDocIds(ints, 0, ints.length, out);
|
||||
} else {
|
||||
docIdsWriter.writeDocIds(ints, 0, ints.length, out);
|
||||
}
|
||||
DocIdsWriter.writeDocIds(ints, 0, ints.length, out);
|
||||
len = out.getFilePointer();
|
||||
if (random().nextBoolean()) {
|
||||
out.writeLong(0); // garbage
|
||||
|
@ -139,13 +102,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() {
|
||||
|
@ -171,64 +134,4 @@ public class TestDocIdsWriter extends LuceneTestCase {
|
|||
}
|
||||
dir.deleteFile("tmp");
|
||||
}
|
||||
|
||||
// This is a fork of legacy DocIdsWriter to test backward compatibility.
|
||||
private static void legacyWriteDocIds(int[] docIds, int start, int count, DataOutput out)
|
||||
throws IOException {
|
||||
boolean sorted = true;
|
||||
for (int i = 1; i < count; ++i) {
|
||||
if (docIds[start + i - 1] > docIds[start + i]) {
|
||||
sorted = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
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;
|
||||
}
|
||||
} else {
|
||||
long max = 0;
|
||||
for (int i = 0; i < count; ++i) {
|
||||
max |= Integer.toUnsignedLong(docIds[start + i]);
|
||||
}
|
||||
if (max <= 0xffffff) {
|
||||
out.writeByte((byte) 24);
|
||||
// write them the same way we are reading them.
|
||||
int i;
|
||||
for (i = 0; i < count - 7; i += 8) {
|
||||
int doc1 = docIds[start + i];
|
||||
int doc2 = docIds[start + i + 1];
|
||||
int doc3 = docIds[start + i + 2];
|
||||
int doc4 = docIds[start + i + 3];
|
||||
int doc5 = docIds[start + i + 4];
|
||||
int doc6 = docIds[start + i + 5];
|
||||
int doc7 = docIds[start + i + 6];
|
||||
int doc8 = docIds[start + i + 7];
|
||||
long l1 = (doc1 & 0xffffffL) << 40 | (doc2 & 0xffffffL) << 16 | ((doc3 >>> 8) & 0xffffL);
|
||||
long l2 =
|
||||
(doc3 & 0xffL) << 56
|
||||
| (doc4 & 0xffffffL) << 32
|
||||
| (doc5 & 0xffffffL) << 8
|
||||
| ((doc6 >> 16) & 0xffL);
|
||||
long l3 = (doc6 & 0xffffL) << 48 | (doc7 & 0xffffffL) << 24 | (doc8 & 0xffffffL);
|
||||
out.writeLong(l1);
|
||||
out.writeLong(l2);
|
||||
out.writeLong(l3);
|
||||
}
|
||||
for (; i < count; ++i) {
|
||||
out.writeShort((short) (docIds[start + i] >>> 8));
|
||||
out.writeByte((byte) docIds[start + i]);
|
||||
}
|
||||
} else {
|
||||
out.writeByte((byte) 32);
|
||||
for (int i = 0; i < count; ++i) {
|
||||
out.writeInt(docIds[start + i]);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -243,65 +243,6 @@ 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