mirror of https://github.com/apache/lucene.git
LUCENE-7563: use a compressed format for the in-heap BKD index
This commit is contained in:
parent
39c2f3d80f
commit
5e8db2e068
|
@ -126,6 +126,10 @@ Optimizations
|
|||
* LUCENE-7568: Optimize merging when index sorting is used but the
|
||||
index is already sorted (Jim Ferenczi via Mike McCandless)
|
||||
|
||||
* LUCENE-7563: The BKD in-memory index for dimensional points now uses
|
||||
a compressed format, using substantially less RAM in some cases
|
||||
(Adrien Grand, Mike McCandless)
|
||||
|
||||
Other
|
||||
|
||||
* LUCENE-7546: Fixed references to benchmark wikipedia data and the Jenkins line-docs file
|
||||
|
|
|
@ -16,13 +16,17 @@
|
|||
*/
|
||||
package org.apache.lucene.codecs.simpletext;
|
||||
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
|
||||
import org.apache.lucene.codecs.simpletext.SimpleTextUtil;
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.PointValues;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.bkd.BKDReader;
|
||||
|
||||
|
@ -30,15 +34,105 @@ import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.BLOCK_C
|
|||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.BLOCK_DOC_ID;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.BLOCK_VALUE;
|
||||
|
||||
class SimpleTextBKDReader extends BKDReader {
|
||||
/** Forked from {@link BKDReader} and simplified/specialized for SimpleText's usage */
|
||||
|
||||
public SimpleTextBKDReader(IndexInput datIn, int numDims, int maxPointsInLeafNode, int bytesPerDim, long[] leafBlockFPs, byte[] splitPackedValues,
|
||||
final class SimpleTextBKDReader extends PointValues implements Accountable {
|
||||
// Packed array of byte[] holding all split values in the full binary tree:
|
||||
final private byte[] splitPackedValues;
|
||||
final long[] leafBlockFPs;
|
||||
final private int leafNodeOffset;
|
||||
final int numDims;
|
||||
final int bytesPerDim;
|
||||
final int bytesPerIndexEntry;
|
||||
final IndexInput in;
|
||||
final int maxPointsInLeafNode;
|
||||
final byte[] minPackedValue;
|
||||
final byte[] maxPackedValue;
|
||||
final long pointCount;
|
||||
final int docCount;
|
||||
final int version;
|
||||
protected final int packedBytesLength;
|
||||
|
||||
public SimpleTextBKDReader(IndexInput in, int numDims, int maxPointsInLeafNode, int bytesPerDim, long[] leafBlockFPs, byte[] splitPackedValues,
|
||||
byte[] minPackedValue, byte[] maxPackedValue, long pointCount, int docCount) throws IOException {
|
||||
super(datIn, numDims, maxPointsInLeafNode, bytesPerDim, leafBlockFPs, splitPackedValues, minPackedValue, maxPackedValue, pointCount, docCount);
|
||||
this.in = in;
|
||||
this.numDims = numDims;
|
||||
this.maxPointsInLeafNode = maxPointsInLeafNode;
|
||||
this.bytesPerDim = bytesPerDim;
|
||||
// no version check here because callers of this API (SimpleText) have no back compat:
|
||||
bytesPerIndexEntry = numDims == 1 ? bytesPerDim : bytesPerDim + 1;
|
||||
packedBytesLength = numDims * bytesPerDim;
|
||||
this.leafNodeOffset = leafBlockFPs.length;
|
||||
this.leafBlockFPs = leafBlockFPs;
|
||||
this.splitPackedValues = splitPackedValues;
|
||||
this.minPackedValue = minPackedValue;
|
||||
this.maxPackedValue = maxPackedValue;
|
||||
this.pointCount = pointCount;
|
||||
this.docCount = docCount;
|
||||
this.version = SimpleTextBKDWriter.VERSION_CURRENT;
|
||||
assert minPackedValue.length == packedBytesLength;
|
||||
assert maxPackedValue.length == packedBytesLength;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void visitDocIDs(IndexInput in, long blockFP, IntersectVisitor visitor) throws IOException {
|
||||
/** Used to track all state for a single call to {@link #intersect}. */
|
||||
public static final class IntersectState {
|
||||
final IndexInput in;
|
||||
final int[] scratchDocIDs;
|
||||
final byte[] scratchPackedValue;
|
||||
final int[] commonPrefixLengths;
|
||||
|
||||
final IntersectVisitor visitor;
|
||||
|
||||
public IntersectState(IndexInput in, int numDims,
|
||||
int packedBytesLength,
|
||||
int maxPointsInLeafNode,
|
||||
IntersectVisitor visitor) {
|
||||
this.in = in;
|
||||
this.visitor = visitor;
|
||||
this.commonPrefixLengths = new int[numDims];
|
||||
this.scratchDocIDs = new int[maxPointsInLeafNode];
|
||||
this.scratchPackedValue = new byte[packedBytesLength];
|
||||
}
|
||||
}
|
||||
|
||||
public void intersect(IntersectVisitor visitor) throws IOException {
|
||||
intersect(getIntersectState(visitor), 1, minPackedValue, maxPackedValue);
|
||||
}
|
||||
|
||||
/** Fast path: this is called when the query box fully encompasses all cells under this node. */
|
||||
private void addAll(IntersectState state, int nodeID) throws IOException {
|
||||
//System.out.println("R: addAll nodeID=" + nodeID);
|
||||
|
||||
if (nodeID >= leafNodeOffset) {
|
||||
//System.out.println("ADDALL");
|
||||
visitDocIDs(state.in, leafBlockFPs[nodeID-leafNodeOffset], state.visitor);
|
||||
// TODO: we can assert that the first value here in fact matches what the index claimed?
|
||||
} else {
|
||||
addAll(state, 2*nodeID);
|
||||
addAll(state, 2*nodeID+1);
|
||||
}
|
||||
}
|
||||
|
||||
/** Create a new {@link IntersectState} */
|
||||
public IntersectState getIntersectState(IntersectVisitor visitor) {
|
||||
return new IntersectState(in.clone(), numDims,
|
||||
packedBytesLength,
|
||||
maxPointsInLeafNode,
|
||||
visitor);
|
||||
}
|
||||
|
||||
/** Visits all docIDs and packed values in a single leaf block */
|
||||
public void visitLeafBlockValues(int nodeID, IntersectState state) throws IOException {
|
||||
int leafID = nodeID - leafNodeOffset;
|
||||
|
||||
// Leaf node; scan and filter all points in this block:
|
||||
int count = readDocIDs(state.in, leafBlockFPs[leafID], state.scratchDocIDs);
|
||||
|
||||
// Again, this time reading values and checking with the visitor
|
||||
visitDocValues(state.commonPrefixLengths, state.scratchPackedValue, state.in, state.scratchDocIDs, count, state.visitor);
|
||||
}
|
||||
|
||||
void visitDocIDs(IndexInput in, long blockFP, IntersectVisitor visitor) throws IOException {
|
||||
BytesRefBuilder scratch = new BytesRefBuilder();
|
||||
in.seek(blockFP);
|
||||
readLine(in, scratch);
|
||||
|
@ -50,8 +144,7 @@ class SimpleTextBKDReader extends BKDReader {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int readDocIDs(IndexInput in, long blockFP, int[] docIDs) throws IOException {
|
||||
int readDocIDs(IndexInput in, long blockFP, int[] docIDs) throws IOException {
|
||||
BytesRefBuilder scratch = new BytesRefBuilder();
|
||||
in.seek(blockFP);
|
||||
readLine(in, scratch);
|
||||
|
@ -63,8 +156,7 @@ class SimpleTextBKDReader extends BKDReader {
|
|||
return count;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void visitDocValues(int[] commonPrefixLengths, byte[] scratchPackedValue, IndexInput in, int[] docIDs, int count, IntersectVisitor visitor) throws IOException {
|
||||
void visitDocValues(int[] commonPrefixLengths, byte[] scratchPackedValue, IndexInput in, int[] docIDs, int count, IntersectVisitor visitor) throws IOException {
|
||||
visitor.grow(count);
|
||||
// NOTE: we don't do prefix coding, so we ignore commonPrefixLengths
|
||||
assert scratchPackedValue.length == packedBytesLength;
|
||||
|
@ -79,6 +171,175 @@ class SimpleTextBKDReader extends BKDReader {
|
|||
}
|
||||
}
|
||||
|
||||
private void visitCompressedDocValues(int[] commonPrefixLengths, byte[] scratchPackedValue, IndexInput in, int[] docIDs, int count, IntersectVisitor visitor, int compressedDim) throws IOException {
|
||||
// the byte at `compressedByteOffset` is compressed using run-length compression,
|
||||
// other suffix bytes are stored verbatim
|
||||
final int compressedByteOffset = compressedDim * bytesPerDim + commonPrefixLengths[compressedDim];
|
||||
commonPrefixLengths[compressedDim]++;
|
||||
int i;
|
||||
for (i = 0; i < count; ) {
|
||||
scratchPackedValue[compressedByteOffset] = in.readByte();
|
||||
final int runLen = Byte.toUnsignedInt(in.readByte());
|
||||
for (int j = 0; j < runLen; ++j) {
|
||||
for(int dim=0;dim<numDims;dim++) {
|
||||
int prefix = commonPrefixLengths[dim];
|
||||
in.readBytes(scratchPackedValue, dim*bytesPerDim + prefix, bytesPerDim - prefix);
|
||||
}
|
||||
visitor.visit(docIDs[i+j], scratchPackedValue);
|
||||
}
|
||||
i += runLen;
|
||||
}
|
||||
if (i != count) {
|
||||
throw new CorruptIndexException("Sub blocks do not add up to the expected count: " + count + " != " + i, in);
|
||||
}
|
||||
}
|
||||
|
||||
private int readCompressedDim(IndexInput in) throws IOException {
|
||||
int compressedDim = in.readByte();
|
||||
if (compressedDim < -1 || compressedDim >= numDims) {
|
||||
throw new CorruptIndexException("Got compressedDim="+compressedDim, in);
|
||||
}
|
||||
return compressedDim;
|
||||
}
|
||||
|
||||
private void readCommonPrefixes(int[] commonPrefixLengths, byte[] scratchPackedValue, IndexInput in) throws IOException {
|
||||
for(int dim=0;dim<numDims;dim++) {
|
||||
int prefix = in.readVInt();
|
||||
commonPrefixLengths[dim] = prefix;
|
||||
if (prefix > 0) {
|
||||
in.readBytes(scratchPackedValue, dim*bytesPerDim, prefix);
|
||||
}
|
||||
//System.out.println("R: " + dim + " of " + numDims + " prefix=" + prefix);
|
||||
}
|
||||
}
|
||||
|
||||
private void intersect(IntersectState state,
|
||||
int nodeID,
|
||||
byte[] cellMinPacked, byte[] cellMaxPacked)
|
||||
throws IOException {
|
||||
|
||||
/*
|
||||
System.out.println("\nR: intersect nodeID=" + nodeID);
|
||||
for(int dim=0;dim<numDims;dim++) {
|
||||
System.out.println(" dim=" + dim + "\n cellMin=" + new BytesRef(cellMinPacked, dim*bytesPerDim, bytesPerDim) + "\n cellMax=" + new BytesRef(cellMaxPacked, dim*bytesPerDim, bytesPerDim));
|
||||
}
|
||||
*/
|
||||
|
||||
Relation r = state.visitor.compare(cellMinPacked, cellMaxPacked);
|
||||
|
||||
if (r == Relation.CELL_OUTSIDE_QUERY) {
|
||||
// This cell is fully outside of the query shape: stop recursing
|
||||
return;
|
||||
} else if (r == Relation.CELL_INSIDE_QUERY) {
|
||||
// This cell is fully inside of the query shape: recursively add all points in this cell without filtering
|
||||
addAll(state, nodeID);
|
||||
return;
|
||||
} else {
|
||||
// The cell crosses the shape boundary, or the cell fully contains the query, so we fall through and do full filtering
|
||||
}
|
||||
|
||||
if (nodeID >= leafNodeOffset) {
|
||||
// TODO: we can assert that the first value here in fact matches what the index claimed?
|
||||
|
||||
int leafID = nodeID - leafNodeOffset;
|
||||
|
||||
// In the unbalanced case it's possible the left most node only has one child:
|
||||
if (leafID < leafBlockFPs.length) {
|
||||
// Leaf node; scan and filter all points in this block:
|
||||
int count = readDocIDs(state.in, leafBlockFPs[leafID], state.scratchDocIDs);
|
||||
|
||||
// Again, this time reading values and checking with the visitor
|
||||
visitDocValues(state.commonPrefixLengths, state.scratchPackedValue, state.in, state.scratchDocIDs, count, state.visitor);
|
||||
}
|
||||
|
||||
} else {
|
||||
|
||||
// Non-leaf node: recurse on the split left and right nodes
|
||||
|
||||
int address = nodeID * bytesPerIndexEntry;
|
||||
int splitDim;
|
||||
if (numDims == 1) {
|
||||
splitDim = 0;
|
||||
} else {
|
||||
splitDim = splitPackedValues[address++] & 0xff;
|
||||
}
|
||||
|
||||
assert splitDim < numDims;
|
||||
|
||||
// TODO: can we alloc & reuse this up front?
|
||||
|
||||
byte[] splitPackedValue = new byte[packedBytesLength];
|
||||
|
||||
// Recurse on left sub-tree:
|
||||
System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, packedBytesLength);
|
||||
System.arraycopy(splitPackedValues, address, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
|
||||
intersect(state,
|
||||
2*nodeID,
|
||||
cellMinPacked, splitPackedValue);
|
||||
|
||||
// Recurse on right sub-tree:
|
||||
System.arraycopy(cellMinPacked, 0, splitPackedValue, 0, packedBytesLength);
|
||||
System.arraycopy(splitPackedValues, address, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
|
||||
intersect(state,
|
||||
2*nodeID+1,
|
||||
splitPackedValue, cellMaxPacked);
|
||||
}
|
||||
}
|
||||
|
||||
/** Copies the split value for this node into the provided byte array */
|
||||
public void copySplitValue(int nodeID, byte[] splitPackedValue) {
|
||||
int address = nodeID * bytesPerIndexEntry;
|
||||
int splitDim;
|
||||
if (numDims == 1) {
|
||||
splitDim = 0;
|
||||
} else {
|
||||
splitDim = splitPackedValues[address++] & 0xff;
|
||||
}
|
||||
|
||||
assert splitDim < numDims;
|
||||
System.arraycopy(splitPackedValues, address, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return RamUsageEstimator.sizeOf(splitPackedValues) +
|
||||
RamUsageEstimator.sizeOf(leafBlockFPs);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getMinPackedValue() {
|
||||
return minPackedValue.clone();
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getMaxPackedValue() {
|
||||
return maxPackedValue.clone();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getNumDimensions() {
|
||||
return numDims;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getBytesPerDimension() {
|
||||
return bytesPerDim;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long size() {
|
||||
return pointCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getDocCount() {
|
||||
return docCount;
|
||||
}
|
||||
|
||||
public boolean isLeafNode(int nodeID) {
|
||||
return nodeID >= leafNodeOffset;
|
||||
}
|
||||
|
||||
private int parseInt(BytesRefBuilder scratch, BytesRef prefix) {
|
||||
assert startsWith(scratch, prefix);
|
||||
return Integer.parseInt(stripPrefix(scratch, prefix));
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -36,7 +36,6 @@ import org.apache.lucene.util.BytesRef;
|
|||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.bkd.BKDReader;
|
||||
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.BLOCK_FP;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.BYTES_PER_DIM;
|
||||
|
@ -58,7 +57,7 @@ class SimpleTextPointsReader extends PointsReader {
|
|||
|
||||
private final IndexInput dataIn;
|
||||
final SegmentReadState readState;
|
||||
final Map<String,BKDReader> readers = new HashMap<>();
|
||||
final Map<String,SimpleTextBKDReader> readers = new HashMap<>();
|
||||
final BytesRefBuilder scratch = new BytesRefBuilder();
|
||||
|
||||
public SimpleTextPointsReader(SegmentReadState readState) throws IOException {
|
||||
|
@ -98,7 +97,7 @@ class SimpleTextPointsReader extends PointsReader {
|
|||
this.readState = readState;
|
||||
}
|
||||
|
||||
private BKDReader initReader(long fp) throws IOException {
|
||||
private SimpleTextBKDReader initReader(long fp) throws IOException {
|
||||
// NOTE: matches what writeIndex does in SimpleTextPointsWriter
|
||||
dataIn.seek(fp);
|
||||
readLine(dataIn);
|
||||
|
|
|
@ -20,7 +20,6 @@ package org.apache.lucene.codecs.simpletext;
|
|||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.function.IntFunction;
|
||||
|
||||
import org.apache.lucene.codecs.PointsReader;
|
||||
import org.apache.lucene.codecs.PointsWriter;
|
||||
|
@ -33,29 +32,28 @@ import org.apache.lucene.index.SegmentWriteState;
|
|||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.apache.lucene.util.bkd.BKDWriter;
|
||||
|
||||
class SimpleTextPointsWriter extends PointsWriter {
|
||||
|
||||
final static BytesRef NUM_DIMS = new BytesRef("num dims ");
|
||||
final static BytesRef BYTES_PER_DIM = new BytesRef("bytes per dim ");
|
||||
final static BytesRef MAX_LEAF_POINTS = new BytesRef("max leaf points ");
|
||||
final static BytesRef INDEX_COUNT = new BytesRef("index count ");
|
||||
final static BytesRef BLOCK_COUNT = new BytesRef("block count ");
|
||||
final static BytesRef BLOCK_DOC_ID = new BytesRef(" doc ");
|
||||
final static BytesRef BLOCK_FP = new BytesRef(" block fp ");
|
||||
final static BytesRef BLOCK_VALUE = new BytesRef(" block value ");
|
||||
final static BytesRef SPLIT_COUNT = new BytesRef("split count ");
|
||||
final static BytesRef SPLIT_DIM = new BytesRef(" split dim ");
|
||||
final static BytesRef SPLIT_VALUE = new BytesRef(" split value ");
|
||||
final static BytesRef FIELD_COUNT = new BytesRef("field count ");
|
||||
final static BytesRef FIELD_FP_NAME = new BytesRef(" field fp name ");
|
||||
final static BytesRef FIELD_FP = new BytesRef(" field fp ");
|
||||
final static BytesRef MIN_VALUE = new BytesRef("min value ");
|
||||
final static BytesRef MAX_VALUE = new BytesRef("max value ");
|
||||
final static BytesRef POINT_COUNT = new BytesRef("point count ");
|
||||
final static BytesRef DOC_COUNT = new BytesRef("doc count ");
|
||||
final static BytesRef END = new BytesRef("END");
|
||||
public final static BytesRef NUM_DIMS = new BytesRef("num dims ");
|
||||
public final static BytesRef BYTES_PER_DIM = new BytesRef("bytes per dim ");
|
||||
public final static BytesRef MAX_LEAF_POINTS = new BytesRef("max leaf points ");
|
||||
public final static BytesRef INDEX_COUNT = new BytesRef("index count ");
|
||||
public final static BytesRef BLOCK_COUNT = new BytesRef("block count ");
|
||||
public final static BytesRef BLOCK_DOC_ID = new BytesRef(" doc ");
|
||||
public final static BytesRef BLOCK_FP = new BytesRef(" block fp ");
|
||||
public final static BytesRef BLOCK_VALUE = new BytesRef(" block value ");
|
||||
public final static BytesRef SPLIT_COUNT = new BytesRef("split count ");
|
||||
public final static BytesRef SPLIT_DIM = new BytesRef(" split dim ");
|
||||
public final static BytesRef SPLIT_VALUE = new BytesRef(" split value ");
|
||||
public final static BytesRef FIELD_COUNT = new BytesRef("field count ");
|
||||
public final static BytesRef FIELD_FP_NAME = new BytesRef(" field fp name ");
|
||||
public final static BytesRef FIELD_FP = new BytesRef(" field fp ");
|
||||
public final static BytesRef MIN_VALUE = new BytesRef("min value ");
|
||||
public final static BytesRef MAX_VALUE = new BytesRef("max value ");
|
||||
public final static BytesRef POINT_COUNT = new BytesRef("point count ");
|
||||
public final static BytesRef DOC_COUNT = new BytesRef("doc count ");
|
||||
public final static BytesRef END = new BytesRef("END");
|
||||
|
||||
private IndexOutput dataOut;
|
||||
final BytesRefBuilder scratch = new BytesRefBuilder();
|
||||
|
@ -75,105 +73,15 @@ class SimpleTextPointsWriter extends PointsWriter {
|
|||
boolean singleValuePerDoc = values.size() == values.getDocCount();
|
||||
|
||||
// We use the normal BKDWriter, but subclass to customize how it writes the index and blocks to disk:
|
||||
try (BKDWriter writer = new BKDWriter(writeState.segmentInfo.maxDoc(),
|
||||
writeState.directory,
|
||||
writeState.segmentInfo.name,
|
||||
fieldInfo.getPointDimensionCount(),
|
||||
fieldInfo.getPointNumBytes(),
|
||||
BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE,
|
||||
BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP,
|
||||
values.size(),
|
||||
singleValuePerDoc) {
|
||||
|
||||
@Override
|
||||
protected void writeIndex(IndexOutput out, long[] leafBlockFPs, byte[] splitPackedValues) throws IOException {
|
||||
write(out, NUM_DIMS);
|
||||
writeInt(out, numDims);
|
||||
newline(out);
|
||||
|
||||
write(out, BYTES_PER_DIM);
|
||||
writeInt(out, bytesPerDim);
|
||||
newline(out);
|
||||
|
||||
write(out, MAX_LEAF_POINTS);
|
||||
writeInt(out, maxPointsInLeafNode);
|
||||
newline(out);
|
||||
|
||||
write(out, INDEX_COUNT);
|
||||
writeInt(out, leafBlockFPs.length);
|
||||
newline(out);
|
||||
|
||||
write(out, MIN_VALUE);
|
||||
BytesRef br = new BytesRef(minPackedValue, 0, minPackedValue.length);
|
||||
write(out, br.toString());
|
||||
newline(out);
|
||||
|
||||
write(out, MAX_VALUE);
|
||||
br = new BytesRef(maxPackedValue, 0, maxPackedValue.length);
|
||||
write(out, br.toString());
|
||||
newline(out);
|
||||
|
||||
write(out, POINT_COUNT);
|
||||
writeLong(out, pointCount);
|
||||
newline(out);
|
||||
|
||||
write(out, DOC_COUNT);
|
||||
writeInt(out, docsSeen.cardinality());
|
||||
newline(out);
|
||||
|
||||
for(int i=0;i<leafBlockFPs.length;i++) {
|
||||
write(out, BLOCK_FP);
|
||||
writeLong(out, leafBlockFPs[i]);
|
||||
newline(out);
|
||||
}
|
||||
|
||||
assert (splitPackedValues.length % (1 + fieldInfo.getPointNumBytes())) == 0;
|
||||
int count = splitPackedValues.length / (1 + fieldInfo.getPointNumBytes());
|
||||
assert count == leafBlockFPs.length;
|
||||
|
||||
write(out, SPLIT_COUNT);
|
||||
writeInt(out, count);
|
||||
newline(out);
|
||||
|
||||
for(int i=0;i<count;i++) {
|
||||
write(out, SPLIT_DIM);
|
||||
writeInt(out, splitPackedValues[i * (1 + fieldInfo.getPointNumBytes())] & 0xff);
|
||||
newline(out);
|
||||
write(out, SPLIT_VALUE);
|
||||
br = new BytesRef(splitPackedValues, 1+(i * (1+fieldInfo.getPointNumBytes())), fieldInfo.getPointNumBytes());
|
||||
write(out, br.toString());
|
||||
newline(out);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void writeLeafBlockDocs(IndexOutput out, int[] docIDs, int start, int count) throws IOException {
|
||||
write(out, BLOCK_COUNT);
|
||||
writeInt(out, count);
|
||||
newline(out);
|
||||
for(int i=0;i<count;i++) {
|
||||
write(out, BLOCK_DOC_ID);
|
||||
writeInt(out, docIDs[start+i]);
|
||||
newline(out);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void writeCommonPrefixes(IndexOutput out, int[] commonPrefixLengths, byte[] packedValue) {
|
||||
// NOTE: we don't do prefix coding, so we ignore commonPrefixLengths
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void writeLeafBlockPackedValues(IndexOutput out, int[] commonPrefixLengths, int count, int sortedDim, IntFunction<BytesRef> packedValues) throws IOException {
|
||||
for (int i = 0; i < count; ++i) {
|
||||
BytesRef packedValue = packedValues.apply(i);
|
||||
// NOTE: we don't do prefix coding, so we ignore commonPrefixLengths
|
||||
write(out, BLOCK_VALUE);
|
||||
write(out, packedValue.toString());
|
||||
newline(out);
|
||||
}
|
||||
}
|
||||
}) {
|
||||
try (SimpleTextBKDWriter writer = new SimpleTextBKDWriter(writeState.segmentInfo.maxDoc(),
|
||||
writeState.directory,
|
||||
writeState.segmentInfo.name,
|
||||
fieldInfo.getPointDimensionCount(),
|
||||
fieldInfo.getPointNumBytes(),
|
||||
SimpleTextBKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE,
|
||||
SimpleTextBKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP,
|
||||
values.size(),
|
||||
singleValuePerDoc)) {
|
||||
|
||||
values.intersect(new IntersectVisitor() {
|
||||
@Override
|
||||
|
@ -198,26 +106,6 @@ class SimpleTextPointsWriter extends PointsWriter {
|
|||
}
|
||||
}
|
||||
|
||||
private void write(IndexOutput out, String s) throws IOException {
|
||||
SimpleTextUtil.write(out, s, scratch);
|
||||
}
|
||||
|
||||
private void writeInt(IndexOutput out, int x) throws IOException {
|
||||
SimpleTextUtil.write(out, Integer.toString(x), scratch);
|
||||
}
|
||||
|
||||
private void writeLong(IndexOutput out, long x) throws IOException {
|
||||
SimpleTextUtil.write(out, Long.toString(x), scratch);
|
||||
}
|
||||
|
||||
private void write(IndexOutput out, BytesRef b) throws IOException {
|
||||
SimpleTextUtil.write(out, b);
|
||||
}
|
||||
|
||||
private void newline(IndexOutput out) throws IOException {
|
||||
SimpleTextUtil.writeNewline(out);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finish() throws IOException {
|
||||
SimpleTextUtil.write(dataOut, END);
|
||||
|
@ -250,4 +138,24 @@ class SimpleTextPointsWriter extends PointsWriter {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void write(IndexOutput out, String s) throws IOException {
|
||||
SimpleTextUtil.write(out, s, scratch);
|
||||
}
|
||||
|
||||
private void writeInt(IndexOutput out, int x) throws IOException {
|
||||
SimpleTextUtil.write(out, Integer.toString(x), scratch);
|
||||
}
|
||||
|
||||
private void writeLong(IndexOutput out, long x) throws IOException {
|
||||
SimpleTextUtil.write(out, Long.toString(x), scratch);
|
||||
}
|
||||
|
||||
private void write(IndexOutput out, BytesRef b) throws IOException {
|
||||
SimpleTextUtil.write(out, b);
|
||||
}
|
||||
|
||||
private void newline(IndexOutput out) throws IOException {
|
||||
SimpleTextUtil.writeNewline(out);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,7 +28,8 @@ import org.apache.lucene.index.SegmentWriteState;
|
|||
|
||||
/**
|
||||
* Lucene 6.0 point format, which encodes dimensional values in a block KD-tree structure
|
||||
* for fast shape intersection filtering. See <a href="https://www.cs.duke.edu/~pankaj/publications/papers/bkd-sstd.pdf">this paper</a> for details.
|
||||
* for fast 1D range and N dimesional shape intersection filtering.
|
||||
* See <a href="https://www.cs.duke.edu/~pankaj/publications/papers/bkd-sstd.pdf">this paper</a> for details.
|
||||
*
|
||||
* <p>This data structure is written as a series of blocks on disk, with an in-memory perfectly balanced
|
||||
* binary tree of split values referencing those blocks at the leaves.
|
||||
|
@ -50,10 +51,13 @@ import org.apache.lucene.index.SegmentWriteState;
|
|||
* <li> maxPointsInLeafNode (vInt)
|
||||
* <li> bytesPerDim (vInt)
|
||||
* <li> count (vInt)
|
||||
* <li> byte[bytesPerDim]<sup>count</sup> (packed <code>byte[]</code> all split values)
|
||||
* <li> delta-blockFP (vLong)<sup>count</sup> (delta-coded file pointers to the on-disk leaf blocks))
|
||||
* <li> packed index (byte[])
|
||||
* </ul>
|
||||
*
|
||||
* <p>The packed index uses hierarchical delta and prefix coding to compactly encode the file pointer for
|
||||
* all leaf blocks, once the tree is traversed, as well as the split dimension and split value for each
|
||||
* inner node of the tree.
|
||||
*
|
||||
* <p>After all fields blocks + index data are written, {@link CodecUtil#writeFooter} writes the checksum.
|
||||
*
|
||||
* <p>The <code>.dii</code> file records the file pointer in the <code>.dim</code> file where each field's
|
||||
|
|
|
@ -16,7 +16,7 @@
|
|||
*/
|
||||
|
||||
/**
|
||||
* Components from the Lucene 6.0 index format. See {@link org.apache.lucene.codecs.lucene62}
|
||||
* for an overview of the index format.
|
||||
* Components from the Lucene 6.0 index format. See {@link org.apache.lucene.codecs.lucene70}
|
||||
* for an overview of the current index format.
|
||||
*/
|
||||
package org.apache.lucene.codecs.lucene60;
|
||||
|
|
|
@ -17,8 +17,8 @@
|
|||
|
||||
/**
|
||||
* Components from the Lucene 6.2 index format
|
||||
* See {@link org.apache.lucene.codecs.lucene62} for an overview
|
||||
* of the index format.
|
||||
* See {@link org.apache.lucene.codecs.lucene70} for an overview
|
||||
* of the current index format.
|
||||
*/
|
||||
|
||||
package org.apache.lucene.codecs.lucene62;
|
||||
|
|
|
@ -185,6 +185,12 @@
|
|||
* {@link org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat Live documents}.
|
||||
* An optional file indicating which documents are live.
|
||||
* </li>
|
||||
* <li>
|
||||
* {@link org.apache.lucene.codecs.lucene60.Lucene60PointsFormat Point values}.
|
||||
* Optional pair of files, recording dimensionally indexed fields, to enable fast
|
||||
* numeric range filtering and large numeric values like BigInteger and BigDecimal (1D)
|
||||
* and geographic shape intersection (2D, 3D).
|
||||
* </li>
|
||||
* </ul>
|
||||
* <p>Details on each of these are provided in their linked pages.</p>
|
||||
* </div>
|
||||
|
@ -300,7 +306,12 @@
|
|||
* <tr>
|
||||
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat Live Documents}</td>
|
||||
* <td>.liv</td>
|
||||
* <td>Info about what files are live</td>
|
||||
* <td>Info about what documents are live</td>
|
||||
* </tr>
|
||||
* <tr>
|
||||
* <td>{@link org.apache.lucene.codecs.lucene60.Lucene60PointsFormat Point values}</td>
|
||||
* <td>.dii, .dim</td>
|
||||
* <td>Holds indexed points, if any</td>
|
||||
* </tr>
|
||||
* </table>
|
||||
* </div>
|
||||
|
@ -374,6 +385,8 @@
|
|||
* that is suitable for faceting/sorting/analytics.
|
||||
* <li>In version 5.4, DocValues have been improved to store more information on disk:
|
||||
* addresses for binary fields and ord indexes for multi-valued fields.
|
||||
* <li>In version 6.0, Points were added, for multi-dimensional range/distance search.
|
||||
* <li>In version 6.2, new Segment info format that reads/writes the index sort, to support index sorting.
|
||||
* <li>In version 7.0, DocValues have been improved to better support sparse doc values
|
||||
* thanks to an iterator API.
|
||||
* </li>
|
||||
|
|
|
@ -1801,161 +1801,32 @@ public final class CheckIndex implements Closeable {
|
|||
}
|
||||
for (FieldInfo fieldInfo : fieldInfos) {
|
||||
if (fieldInfo.getPointDimensionCount() > 0) {
|
||||
FixedBitSet docsSeen = new FixedBitSet(reader.maxDoc());
|
||||
status.totalValueFields++;
|
||||
int dimCount = fieldInfo.getPointDimensionCount();
|
||||
int bytesPerDim = fieldInfo.getPointNumBytes();
|
||||
int packedBytesCount = dimCount * bytesPerDim;
|
||||
byte[] lastMinPackedValue = new byte[packedBytesCount];
|
||||
byte[] lastMaxPackedValue = new byte[packedBytesCount];
|
||||
BytesRef scratch = new BytesRef();
|
||||
scratch.length = bytesPerDim;
|
||||
byte[] lastPackedValue = new byte[packedBytesCount];
|
||||
|
||||
long[] pointCountSeen = new long[1];
|
||||
|
||||
PointValues values = pointsReader.getValues(fieldInfo.name);
|
||||
if (values == null) {
|
||||
continue;
|
||||
}
|
||||
byte[] globalMinPackedValue = values.getMinPackedValue();
|
||||
|
||||
status.totalValueFields++;
|
||||
|
||||
long size = values.size();
|
||||
int docCount = values.getDocCount();
|
||||
|
||||
if (docCount > size) {
|
||||
throw new RuntimeException("point values for field \"" + fieldInfo.name + "\" claims to have size=" + size + " points and inconsistent docCount=" + docCount);
|
||||
VerifyPointsVisitor visitor = new VerifyPointsVisitor(fieldInfo.name, reader.maxDoc(), values);
|
||||
values.intersect(visitor);
|
||||
|
||||
if (visitor.getPointCountSeen() != size) {
|
||||
throw new RuntimeException("point values for field \"" + fieldInfo.name + "\" claims to have size=" + size + " points, but in fact has " + visitor.getPointCountSeen());
|
||||
}
|
||||
|
||||
if (docCount > reader.maxDoc()) {
|
||||
throw new RuntimeException("point values for field \"" + fieldInfo.name + "\" claims to have docCount=" + docCount + " but that's greater than maxDoc=" + reader.maxDoc());
|
||||
if (visitor.getDocCountSeen() != docCount) {
|
||||
throw new RuntimeException("point values for field \"" + fieldInfo.name + "\" claims to have docCount=" + docCount + " but in fact has " + visitor.getDocCountSeen());
|
||||
}
|
||||
|
||||
if (globalMinPackedValue == null) {
|
||||
if (size != 0) {
|
||||
throw new RuntimeException("getMinPackedValue is null points for field \"" + fieldInfo.name + "\" yet size=" + size);
|
||||
}
|
||||
} else if (globalMinPackedValue.length != packedBytesCount) {
|
||||
throw new RuntimeException("getMinPackedValue for field \"" + fieldInfo.name + "\" return length=" + globalMinPackedValue.length + " array, but should be " + packedBytesCount);
|
||||
}
|
||||
byte[] globalMaxPackedValue = values.getMaxPackedValue();
|
||||
if (globalMaxPackedValue == null) {
|
||||
if (size != 0) {
|
||||
throw new RuntimeException("getMaxPackedValue is null points for field \"" + fieldInfo.name + "\" yet size=" + size);
|
||||
}
|
||||
} else if (globalMaxPackedValue.length != packedBytesCount) {
|
||||
throw new RuntimeException("getMaxPackedValue for field \"" + fieldInfo.name + "\" return length=" + globalMaxPackedValue.length + " array, but should be " + packedBytesCount);
|
||||
}
|
||||
|
||||
values.intersect(new PointValues.IntersectVisitor() {
|
||||
|
||||
private int lastDocID = -1;
|
||||
|
||||
@Override
|
||||
public void visit(int docID) {
|
||||
throw new RuntimeException("codec called IntersectVisitor.visit without a packed value for docID=" + docID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visit(int docID, byte[] packedValue) {
|
||||
checkPackedValue("packed value", packedValue, docID);
|
||||
pointCountSeen[0]++;
|
||||
docsSeen.set(docID);
|
||||
|
||||
for(int dim=0;dim<dimCount;dim++) {
|
||||
int offset = bytesPerDim * dim;
|
||||
|
||||
// Compare to last cell:
|
||||
if (StringHelper.compare(bytesPerDim, packedValue, offset, lastMinPackedValue, offset) < 0) {
|
||||
// This doc's point, in this dimension, is lower than the minimum value of the last cell checked:
|
||||
throw new RuntimeException("packed points value " + Arrays.toString(packedValue) + " for field=\"" + fieldInfo.name + "\", docID=" + docID + " is out-of-bounds of the last cell min=" + Arrays.toString(lastMinPackedValue) + " max=" + Arrays.toString(lastMaxPackedValue) + " dim=" + dim);
|
||||
}
|
||||
|
||||
if (StringHelper.compare(bytesPerDim, packedValue, offset, lastMaxPackedValue, offset) > 0) {
|
||||
// This doc's point, in this dimension, is greater than the maximum value of the last cell checked:
|
||||
throw new RuntimeException("packed points value " + Arrays.toString(packedValue) + " for field=\"" + fieldInfo.name + "\", docID=" + docID + " is out-of-bounds of the last cell min=" + Arrays.toString(lastMinPackedValue) + " max=" + Arrays.toString(lastMaxPackedValue) + " dim=" + dim);
|
||||
}
|
||||
}
|
||||
|
||||
// In the 1D case, PointValues must make a single in-order sweep through all values, and tie-break by
|
||||
// increasing docID:
|
||||
if (dimCount == 1) {
|
||||
int cmp = StringHelper.compare(bytesPerDim, lastPackedValue, 0, packedValue, 0);
|
||||
if (cmp > 0) {
|
||||
throw new RuntimeException("packed points value " + Arrays.toString(packedValue) + " for field=\"" + fieldInfo.name + "\", for docID=" + docID + " is out-of-order vs the previous document's value " + Arrays.toString(lastPackedValue));
|
||||
} else if (cmp == 0) {
|
||||
if (docID < lastDocID) {
|
||||
throw new RuntimeException("packed points value is the same, but docID=" + docID + " is out of order vs previous docID=" + lastDocID + ", field=\"" + fieldInfo.name + "\"");
|
||||
}
|
||||
}
|
||||
System.arraycopy(packedValue, 0, lastPackedValue, 0, bytesPerDim);
|
||||
lastDocID = docID;
|
||||
}
|
||||
|
||||
status.totalValuePoints++;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PointValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
checkPackedValue("min packed value", minPackedValue, -1);
|
||||
System.arraycopy(minPackedValue, 0, lastMinPackedValue, 0, packedBytesCount);
|
||||
checkPackedValue("max packed value", maxPackedValue, -1);
|
||||
System.arraycopy(maxPackedValue, 0, lastMaxPackedValue, 0, packedBytesCount);
|
||||
|
||||
for(int dim=0;dim<dimCount;dim++) {
|
||||
int offset = bytesPerDim * dim;
|
||||
|
||||
if (StringHelper.compare(bytesPerDim, minPackedValue, offset, maxPackedValue, offset) > 0) {
|
||||
throw new RuntimeException("packed points cell minPackedValue " + Arrays.toString(minPackedValue) +
|
||||
" is out-of-bounds of the cell's maxPackedValue " + Arrays.toString(maxPackedValue) + " dim=" + dim + " field=\"" + fieldInfo.name + "\"");
|
||||
}
|
||||
|
||||
// Make sure this cell is not outside of the global min/max:
|
||||
if (StringHelper.compare(bytesPerDim, minPackedValue, offset, globalMinPackedValue, offset) < 0) {
|
||||
throw new RuntimeException("packed points cell minPackedValue " + Arrays.toString(minPackedValue) +
|
||||
" is out-of-bounds of the global minimum " + Arrays.toString(globalMinPackedValue) + " dim=" + dim + " field=\"" + fieldInfo.name + "\"");
|
||||
}
|
||||
|
||||
if (StringHelper.compare(bytesPerDim, maxPackedValue, offset, globalMinPackedValue, offset) < 0) {
|
||||
throw new RuntimeException("packed points cell maxPackedValue " + Arrays.toString(maxPackedValue) +
|
||||
" is out-of-bounds of the global minimum " + Arrays.toString(globalMinPackedValue) + " dim=" + dim + " field=\"" + fieldInfo.name + "\"");
|
||||
}
|
||||
|
||||
if (StringHelper.compare(bytesPerDim, minPackedValue, offset, globalMaxPackedValue, offset) > 0) {
|
||||
throw new RuntimeException("packed points cell minPackedValue " + Arrays.toString(minPackedValue) +
|
||||
" is out-of-bounds of the global maximum " + Arrays.toString(globalMaxPackedValue) + " dim=" + dim + " field=\"" + fieldInfo.name + "\"");
|
||||
}
|
||||
if (StringHelper.compare(bytesPerDim, maxPackedValue, offset, globalMaxPackedValue, offset) > 0) {
|
||||
throw new RuntimeException("packed points cell maxPackedValue " + Arrays.toString(maxPackedValue) +
|
||||
" is out-of-bounds of the global maximum " + Arrays.toString(globalMaxPackedValue) + " dim=" + dim + " field=\"" + fieldInfo.name + "\"");
|
||||
}
|
||||
}
|
||||
|
||||
// We always pretend the query shape is so complex that it crosses every cell, so
|
||||
// that packedValue is passed for every document
|
||||
return PointValues.Relation.CELL_CROSSES_QUERY;
|
||||
}
|
||||
|
||||
private void checkPackedValue(String desc, byte[] packedValue, int docID) {
|
||||
if (packedValue == null) {
|
||||
throw new RuntimeException(desc + " is null for docID=" + docID + " field=\"" + fieldInfo.name + "\"");
|
||||
}
|
||||
|
||||
if (packedValue.length != packedBytesCount) {
|
||||
throw new RuntimeException(desc + " has incorrect length=" + packedValue.length + " vs expected=" + packedBytesCount + " for docID=" + docID + " field=\"" + fieldInfo.name + "\"");
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
if (pointCountSeen[0] != size) {
|
||||
throw new RuntimeException("point values for field \"" + fieldInfo.name + "\" claims to have size=" + size + " points, but in fact has " + pointCountSeen[0]);
|
||||
}
|
||||
|
||||
if (docsSeen.cardinality() != docCount) {
|
||||
throw new RuntimeException("point values for field \"" + fieldInfo.name + "\" claims to have docCount=" + docCount + " but in fact has " + docsSeen.cardinality());
|
||||
}
|
||||
status.totalValuePoints += visitor.getPointCountSeen();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
msg(infoStream, String.format(Locale.ROOT, "OK [%d fields, %d points] [took %.3f sec]", status.totalValueFields, status.totalValuePoints, nsToSec(System.nanoTime()-startNS)));
|
||||
|
||||
} catch (Throwable e) {
|
||||
|
@ -1972,6 +1843,167 @@ public final class CheckIndex implements Closeable {
|
|||
return status;
|
||||
}
|
||||
|
||||
/** Walks the entire N-dimensional points space, verifying that all points fall within the last cell's boundaries.
|
||||
*
|
||||
* @lucene.internal */
|
||||
public static class VerifyPointsVisitor implements PointValues.IntersectVisitor {
|
||||
private long pointCountSeen;
|
||||
private int lastDocID = -1;
|
||||
private final int maxDoc;
|
||||
private final FixedBitSet docsSeen;
|
||||
private final byte[] lastMinPackedValue;
|
||||
private final byte[] lastMaxPackedValue;
|
||||
private final byte[] lastPackedValue;
|
||||
private final byte[] globalMinPackedValue;
|
||||
private final byte[] globalMaxPackedValue;
|
||||
private final int packedBytesCount;
|
||||
private final int numDims;
|
||||
private final int bytesPerDim;
|
||||
private final String fieldName;
|
||||
|
||||
/** Sole constructor */
|
||||
public VerifyPointsVisitor(String fieldName, int maxDoc, PointValues values) throws IOException {
|
||||
this.maxDoc = maxDoc;
|
||||
this.fieldName = fieldName;
|
||||
numDims = values.getNumDimensions();
|
||||
bytesPerDim = values.getBytesPerDimension();
|
||||
packedBytesCount = numDims * bytesPerDim;
|
||||
globalMinPackedValue = values.getMinPackedValue();
|
||||
globalMaxPackedValue = values.getMaxPackedValue();
|
||||
docsSeen = new FixedBitSet(maxDoc);
|
||||
lastMinPackedValue = new byte[packedBytesCount];
|
||||
lastMaxPackedValue = new byte[packedBytesCount];
|
||||
lastPackedValue = new byte[packedBytesCount];
|
||||
|
||||
if (values.getDocCount() > values.size()) {
|
||||
throw new RuntimeException("point values for field \"" + fieldName + "\" claims to have size=" + values.size() + " points and inconsistent docCount=" + values.getDocCount());
|
||||
}
|
||||
|
||||
if (values.getDocCount() > maxDoc) {
|
||||
throw new RuntimeException("point values for field \"" + fieldName + "\" claims to have docCount=" + values.getDocCount() + " but that's greater than maxDoc=" + maxDoc);
|
||||
}
|
||||
|
||||
if (globalMinPackedValue == null) {
|
||||
if (values.size() != 0) {
|
||||
throw new RuntimeException("getMinPackedValue is null points for field \"" + fieldName + "\" yet size=" + values.size());
|
||||
}
|
||||
} else if (globalMinPackedValue.length != packedBytesCount) {
|
||||
throw new RuntimeException("getMinPackedValue for field \"" + fieldName + "\" return length=" + globalMinPackedValue.length + " array, but should be " + packedBytesCount);
|
||||
}
|
||||
if (globalMaxPackedValue == null) {
|
||||
if (values.size() != 0) {
|
||||
throw new RuntimeException("getMaxPackedValue is null points for field \"" + fieldName + "\" yet size=" + values.size());
|
||||
}
|
||||
} else if (globalMaxPackedValue.length != packedBytesCount) {
|
||||
throw new RuntimeException("getMaxPackedValue for field \"" + fieldName + "\" return length=" + globalMaxPackedValue.length + " array, but should be " + packedBytesCount);
|
||||
}
|
||||
}
|
||||
|
||||
/** Returns total number of points in this BKD tree */
|
||||
public long getPointCountSeen() {
|
||||
return pointCountSeen;
|
||||
}
|
||||
|
||||
/** Returns total number of unique docIDs in this BKD tree */
|
||||
public long getDocCountSeen() {
|
||||
return docsSeen.cardinality();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visit(int docID) {
|
||||
throw new RuntimeException("codec called IntersectVisitor.visit without a packed value for docID=" + docID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visit(int docID, byte[] packedValue) {
|
||||
checkPackedValue("packed value", packedValue, docID);
|
||||
pointCountSeen++;
|
||||
docsSeen.set(docID);
|
||||
|
||||
for(int dim=0;dim<numDims;dim++) {
|
||||
int offset = bytesPerDim * dim;
|
||||
|
||||
// Compare to last cell:
|
||||
if (StringHelper.compare(bytesPerDim, packedValue, offset, lastMinPackedValue, offset) < 0) {
|
||||
// This doc's point, in this dimension, is lower than the minimum value of the last cell checked:
|
||||
throw new RuntimeException("packed points value " + Arrays.toString(packedValue) + " for field=\"" + fieldName + "\", docID=" + docID + " is out-of-bounds of the last cell min=" + Arrays.toString(lastMinPackedValue) + " max=" + Arrays.toString(lastMaxPackedValue) + " dim=" + dim);
|
||||
}
|
||||
|
||||
if (StringHelper.compare(bytesPerDim, packedValue, offset, lastMaxPackedValue, offset) > 0) {
|
||||
// This doc's point, in this dimension, is greater than the maximum value of the last cell checked:
|
||||
throw new RuntimeException("packed points value " + Arrays.toString(packedValue) + " for field=\"" + fieldName + "\", docID=" + docID + " is out-of-bounds of the last cell min=" + Arrays.toString(lastMinPackedValue) + " max=" + Arrays.toString(lastMaxPackedValue) + " dim=" + dim);
|
||||
}
|
||||
}
|
||||
|
||||
// In the 1D case, PointValues must make a single in-order sweep through all values, and tie-break by
|
||||
// increasing docID:
|
||||
if (numDims == 1) {
|
||||
int cmp = StringHelper.compare(bytesPerDim, lastPackedValue, 0, packedValue, 0);
|
||||
if (cmp > 0) {
|
||||
throw new RuntimeException("packed points value " + Arrays.toString(packedValue) + " for field=\"" + fieldName + "\", for docID=" + docID + " is out-of-order vs the previous document's value " + Arrays.toString(lastPackedValue));
|
||||
} else if (cmp == 0) {
|
||||
if (docID < lastDocID) {
|
||||
throw new RuntimeException("packed points value is the same, but docID=" + docID + " is out of order vs previous docID=" + lastDocID + ", field=\"" + fieldName + "\"");
|
||||
}
|
||||
}
|
||||
System.arraycopy(packedValue, 0, lastPackedValue, 0, bytesPerDim);
|
||||
lastDocID = docID;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public PointValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
checkPackedValue("min packed value", minPackedValue, -1);
|
||||
System.arraycopy(minPackedValue, 0, lastMinPackedValue, 0, packedBytesCount);
|
||||
checkPackedValue("max packed value", maxPackedValue, -1);
|
||||
System.arraycopy(maxPackedValue, 0, lastMaxPackedValue, 0, packedBytesCount);
|
||||
|
||||
for(int dim=0;dim<numDims;dim++) {
|
||||
int offset = bytesPerDim * dim;
|
||||
|
||||
if (StringHelper.compare(bytesPerDim, minPackedValue, offset, maxPackedValue, offset) > 0) {
|
||||
throw new RuntimeException("packed points cell minPackedValue " + Arrays.toString(minPackedValue) +
|
||||
" is out-of-bounds of the cell's maxPackedValue " + Arrays.toString(maxPackedValue) + " dim=" + dim + " field=\"" + fieldName + "\"");
|
||||
}
|
||||
|
||||
// Make sure this cell is not outside of the global min/max:
|
||||
if (StringHelper.compare(bytesPerDim, minPackedValue, offset, globalMinPackedValue, offset) < 0) {
|
||||
throw new RuntimeException("packed points cell minPackedValue " + Arrays.toString(minPackedValue) +
|
||||
" is out-of-bounds of the global minimum " + Arrays.toString(globalMinPackedValue) + " dim=" + dim + " field=\"" + fieldName + "\"");
|
||||
}
|
||||
|
||||
if (StringHelper.compare(bytesPerDim, maxPackedValue, offset, globalMinPackedValue, offset) < 0) {
|
||||
throw new RuntimeException("packed points cell maxPackedValue " + Arrays.toString(maxPackedValue) +
|
||||
" is out-of-bounds of the global minimum " + Arrays.toString(globalMinPackedValue) + " dim=" + dim + " field=\"" + fieldName + "\"");
|
||||
}
|
||||
|
||||
if (StringHelper.compare(bytesPerDim, minPackedValue, offset, globalMaxPackedValue, offset) > 0) {
|
||||
throw new RuntimeException("packed points cell minPackedValue " + Arrays.toString(minPackedValue) +
|
||||
" is out-of-bounds of the global maximum " + Arrays.toString(globalMaxPackedValue) + " dim=" + dim + " field=\"" + fieldName + "\"");
|
||||
}
|
||||
if (StringHelper.compare(bytesPerDim, maxPackedValue, offset, globalMaxPackedValue, offset) > 0) {
|
||||
throw new RuntimeException("packed points cell maxPackedValue " + Arrays.toString(maxPackedValue) +
|
||||
" is out-of-bounds of the global maximum " + Arrays.toString(globalMaxPackedValue) + " dim=" + dim + " field=\"" + fieldName + "\"");
|
||||
}
|
||||
}
|
||||
|
||||
// We always pretend the query shape is so complex that it crosses every cell, so
|
||||
// that packedValue is passed for every document
|
||||
return PointValues.Relation.CELL_CROSSES_QUERY;
|
||||
}
|
||||
|
||||
private void checkPackedValue(String desc, byte[] packedValue, int docID) {
|
||||
if (packedValue == null) {
|
||||
throw new RuntimeException(desc + " is null for docID=" + docID + " field=\"" + fieldName + "\"");
|
||||
}
|
||||
|
||||
if (packedValue.length != packedBytesCount) {
|
||||
throw new RuntimeException(desc + " has incorrect length=" + packedValue.length + " vs expected=" + packedBytesCount + " for docID=" + docID + " field=\"" + fieldName + "\"");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Test stored fields.
|
||||
* @lucene.experimental
|
||||
|
|
|
@ -17,14 +17,15 @@
|
|||
package org.apache.lucene.util.bkd;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.PointValues;
|
||||
import org.apache.lucene.store.ByteArrayDataInput;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.MathUtil;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
|
||||
|
@ -32,14 +33,12 @@ import org.apache.lucene.util.StringHelper;
|
|||
*
|
||||
* @lucene.experimental */
|
||||
|
||||
public class BKDReader extends PointValues implements Accountable {
|
||||
public final class BKDReader extends PointValues implements Accountable {
|
||||
// Packed array of byte[] holding all split values in the full binary tree:
|
||||
final private byte[] splitPackedValues;
|
||||
final long[] leafBlockFPs;
|
||||
final private int leafNodeOffset;
|
||||
final int leafNodeOffset;
|
||||
final int numDims;
|
||||
final int bytesPerDim;
|
||||
final int bytesPerIndexEntry;
|
||||
final int numLeaves;
|
||||
final IndexInput in;
|
||||
final int maxPointsInLeafNode;
|
||||
final byte[] minPackedValue;
|
||||
|
@ -49,6 +48,14 @@ public class BKDReader extends PointValues implements Accountable {
|
|||
final int version;
|
||||
protected final int packedBytesLength;
|
||||
|
||||
// Used for 6.4.0+ index format:
|
||||
final byte[] packedIndex;
|
||||
|
||||
// Used for Legacy (pre-6.4.0) index format, to hold a compact form of the index:
|
||||
final private byte[] splitPackedValues;
|
||||
final int bytesPerIndexEntry;
|
||||
final long[] leafBlockFPs;
|
||||
|
||||
/** Caller must pre-seek the provided {@link IndexInput} to the index location that {@link BKDWriter#finish} returned */
|
||||
public BKDReader(IndexInput in) throws IOException {
|
||||
version = CodecUtil.checkHeader(in, BKDWriter.CODEC_NAME, BKDWriter.VERSION_START, BKDWriter.VERSION_CURRENT);
|
||||
|
@ -59,7 +66,7 @@ public class BKDReader extends PointValues implements Accountable {
|
|||
packedBytesLength = numDims * bytesPerDim;
|
||||
|
||||
// Read index:
|
||||
int numLeaves = in.readVInt();
|
||||
numLeaves = in.readVInt();
|
||||
assert numLeaves > 0;
|
||||
leafNodeOffset = numLeaves;
|
||||
|
||||
|
@ -78,203 +85,378 @@ public class BKDReader extends PointValues implements Accountable {
|
|||
pointCount = in.readVLong();
|
||||
docCount = in.readVInt();
|
||||
|
||||
splitPackedValues = new byte[bytesPerIndexEntry*numLeaves];
|
||||
|
||||
// TODO: don't write split packed values[0]!
|
||||
in.readBytes(splitPackedValues, 0, splitPackedValues.length);
|
||||
|
||||
// Read the file pointers to the start of each leaf block:
|
||||
long[] leafBlockFPs = new long[numLeaves];
|
||||
long lastFP = 0;
|
||||
for(int i=0;i<numLeaves;i++) {
|
||||
long delta = in.readVLong();
|
||||
leafBlockFPs[i] = lastFP + delta;
|
||||
lastFP += delta;
|
||||
}
|
||||
|
||||
// Possibly rotate the leaf block FPs, if the index not fully balanced binary tree (only happens
|
||||
// if it was created by BKDWriter.merge). In this case the leaf nodes may straddle the two bottom
|
||||
// levels of the binary tree:
|
||||
if (numDims == 1 && numLeaves > 1) {
|
||||
//System.out.println("BKDR: numLeaves=" + numLeaves);
|
||||
int levelCount = 2;
|
||||
while (true) {
|
||||
//System.out.println(" cycle levelCount=" + levelCount);
|
||||
if (numLeaves >= levelCount && numLeaves <= 2*levelCount) {
|
||||
int lastLevel = 2*(numLeaves - levelCount);
|
||||
assert lastLevel >= 0;
|
||||
/*
|
||||
System.out.println("BKDR: lastLevel=" + lastLevel + " vs " + levelCount);
|
||||
System.out.println("FPs before:");
|
||||
for(int i=0;i<leafBlockFPs.length;i++) {
|
||||
System.out.println(" " + i + " " + leafBlockFPs[i]);
|
||||
}
|
||||
*/
|
||||
if (lastLevel != 0) {
|
||||
// Last level is partially filled, so we must rotate the leaf FPs to match. We do this here, after loading
|
||||
// at read-time, so that we can still delta code them on disk at write:
|
||||
//System.out.println("BKDR: now rotate index");
|
||||
long[] newLeafBlockFPs = new long[numLeaves];
|
||||
System.arraycopy(leafBlockFPs, lastLevel, newLeafBlockFPs, 0, leafBlockFPs.length - lastLevel);
|
||||
System.arraycopy(leafBlockFPs, 0, newLeafBlockFPs, leafBlockFPs.length - lastLevel, lastLevel);
|
||||
leafBlockFPs = newLeafBlockFPs;
|
||||
}
|
||||
/*
|
||||
System.out.println("FPs:");
|
||||
for(int i=0;i<leafBlockFPs.length;i++) {
|
||||
System.out.println(" " + i + " " + leafBlockFPs[i]);
|
||||
}
|
||||
*/
|
||||
break;
|
||||
}
|
||||
|
||||
levelCount *= 2;
|
||||
}
|
||||
}
|
||||
|
||||
this.leafBlockFPs = leafBlockFPs;
|
||||
this.in = in;
|
||||
}
|
||||
|
||||
/** Called by consumers that have their own on-disk format for the index (e.g. SimpleText) */
|
||||
protected BKDReader(IndexInput in, int numDims, int maxPointsInLeafNode, int bytesPerDim, long[] leafBlockFPs, byte[] splitPackedValues,
|
||||
byte[] minPackedValue, byte[] maxPackedValue, long pointCount, int docCount) throws IOException {
|
||||
this.in = in;
|
||||
this.numDims = numDims;
|
||||
this.maxPointsInLeafNode = maxPointsInLeafNode;
|
||||
this.bytesPerDim = bytesPerDim;
|
||||
// no version check here because callers of this API (SimpleText) have no back compat:
|
||||
bytesPerIndexEntry = numDims == 1 ? bytesPerDim : bytesPerDim + 1;
|
||||
packedBytesLength = numDims * bytesPerDim;
|
||||
this.leafNodeOffset = leafBlockFPs.length;
|
||||
this.leafBlockFPs = leafBlockFPs;
|
||||
this.splitPackedValues = splitPackedValues;
|
||||
this.minPackedValue = minPackedValue;
|
||||
this.maxPackedValue = maxPackedValue;
|
||||
this.pointCount = pointCount;
|
||||
this.docCount = docCount;
|
||||
this.version = BKDWriter.VERSION_CURRENT;
|
||||
assert minPackedValue.length == packedBytesLength;
|
||||
assert maxPackedValue.length == packedBytesLength;
|
||||
}
|
||||
|
||||
private static class VerifyVisitor implements IntersectVisitor {
|
||||
byte[] cellMinPacked;
|
||||
byte[] cellMaxPacked;
|
||||
byte[] lastPackedValue;
|
||||
final int numDims;
|
||||
final int bytesPerDim;
|
||||
final int maxDoc;
|
||||
|
||||
public VerifyVisitor(int numDims, int bytesPerDim, int maxDoc) {
|
||||
this.numDims = numDims;
|
||||
this.bytesPerDim = bytesPerDim;
|
||||
this.maxDoc = maxDoc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visit(int docID) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visit(int docID, byte[] packedValue) {
|
||||
if (docID < 0 || docID >= maxDoc) {
|
||||
throw new RuntimeException("docID=" + docID + " is out of bounds of 0.." + maxDoc);
|
||||
}
|
||||
for(int dim=0;dim<numDims;dim++) {
|
||||
if (StringHelper.compare(bytesPerDim, cellMinPacked, dim*bytesPerDim, packedValue, dim*bytesPerDim) > 0) {
|
||||
throw new RuntimeException("value=" + new BytesRef(packedValue, dim*bytesPerDim, bytesPerDim) + " for docID=" + docID + " dim=" + dim + " is less than this leaf block's minimum=" + new BytesRef(cellMinPacked, dim*bytesPerDim, bytesPerDim));
|
||||
}
|
||||
if (StringHelper.compare(bytesPerDim, cellMaxPacked, dim*bytesPerDim, packedValue, dim*bytesPerDim) < 0) {
|
||||
throw new RuntimeException("value=" + new BytesRef(packedValue, dim*bytesPerDim, bytesPerDim) + " for docID=" + docID + " dim=" + dim + " is greater than this leaf block's maximum=" + new BytesRef(cellMaxPacked, dim*bytesPerDim, bytesPerDim));
|
||||
}
|
||||
}
|
||||
|
||||
if (numDims == 1) {
|
||||
// With only 1D, all values should always be in sorted order
|
||||
if (lastPackedValue == null) {
|
||||
lastPackedValue = Arrays.copyOf(packedValue, packedValue.length);
|
||||
} else if (StringHelper.compare(bytesPerDim, lastPackedValue, 0, packedValue, 0) > 0) {
|
||||
throw new RuntimeException("value=" + new BytesRef(packedValue) + " for docID=" + docID + " dim=0" + " sorts before last value=" + new BytesRef(lastPackedValue));
|
||||
} else {
|
||||
System.arraycopy(packedValue, 0, lastPackedValue, 0, bytesPerDim);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
||||
|
||||
/** Only used for debugging, to make sure all values in each leaf block fall within the range expected by the index */
|
||||
// TODO: maybe we can get this into CheckIndex?
|
||||
public void verify(int maxDoc) throws IOException {
|
||||
//System.out.println("BKDR.verify this=" + this);
|
||||
// Visits every doc in every leaf block and confirms that
|
||||
// their values agree with the index:
|
||||
byte[] rootMinPacked = new byte[packedBytesLength];
|
||||
byte[] rootMaxPacked = new byte[packedBytesLength];
|
||||
Arrays.fill(rootMaxPacked, (byte) 0xff);
|
||||
verify(getIntersectState(new VerifyVisitor(numDims, bytesPerDim, maxDoc)), 1, rootMinPacked, rootMaxPacked);
|
||||
}
|
||||
|
||||
private void verify(IntersectState state, int nodeID, byte[] cellMinPacked, byte[] cellMaxPacked) throws IOException {
|
||||
|
||||
if (nodeID >= leafNodeOffset) {
|
||||
int leafID = nodeID - leafNodeOffset;
|
||||
|
||||
// In the unbalanced case it's possible the left most node only has one child:
|
||||
if (leafID < leafBlockFPs.length) {
|
||||
//System.out.println("CHECK nodeID=" + nodeID + " leaf=" + (nodeID-leafNodeOffset) + " offset=" + leafNodeOffset + " fp=" + leafBlockFPs[leafID]);
|
||||
//System.out.println("BKDR.verify leafID=" + leafID + " nodeID=" + nodeID + " fp=" + leafBlockFPs[leafID] + " min=" + new BytesRef(cellMinPacked) + " max=" + new BytesRef(cellMaxPacked));
|
||||
|
||||
// Leaf node: check that all values are in fact in bounds:
|
||||
VerifyVisitor visitor = (VerifyVisitor) state.visitor;
|
||||
visitor.cellMinPacked = cellMinPacked;
|
||||
visitor.cellMaxPacked = cellMaxPacked;
|
||||
|
||||
int count = readDocIDs(state.in, leafBlockFPs[leafID], state.scratchDocIDs);
|
||||
visitDocValues(state.commonPrefixLengths, state.scratchPackedValue, state.in, state.scratchDocIDs, count, state.visitor);
|
||||
} else {
|
||||
//System.out.println("BKDR.verify skip leafID=" + leafID);
|
||||
}
|
||||
if (version >= BKDWriter.VERSION_PACKED_INDEX) {
|
||||
int numBytes = in.readVInt();
|
||||
packedIndex = new byte[numBytes];
|
||||
in.readBytes(packedIndex, 0, numBytes);
|
||||
leafBlockFPs = null;
|
||||
splitPackedValues = null;
|
||||
} else {
|
||||
// Non-leaf node:
|
||||
// legacy un-packed index
|
||||
|
||||
int address = nodeID * bytesPerIndexEntry;
|
||||
int splitDim;
|
||||
if (numDims == 1) {
|
||||
splitDim = 0;
|
||||
if (version < BKDWriter.VERSION_IMPLICIT_SPLIT_DIM_1D) {
|
||||
// skip over wastefully encoded 0 splitDim:
|
||||
assert splitPackedValues[address] == 0;
|
||||
address++;
|
||||
splitPackedValues = new byte[bytesPerIndexEntry*numLeaves];
|
||||
|
||||
in.readBytes(splitPackedValues, 0, splitPackedValues.length);
|
||||
|
||||
// Read the file pointers to the start of each leaf block:
|
||||
long[] leafBlockFPs = new long[numLeaves];
|
||||
long lastFP = 0;
|
||||
for(int i=0;i<numLeaves;i++) {
|
||||
long delta = in.readVLong();
|
||||
leafBlockFPs[i] = lastFP + delta;
|
||||
lastFP += delta;
|
||||
}
|
||||
|
||||
// Possibly rotate the leaf block FPs, if the index not fully balanced binary tree (only happens
|
||||
// if it was created by BKDWriter.merge or OneDimWriter). In this case the leaf nodes may straddle the two bottom
|
||||
// levels of the binary tree:
|
||||
if (numDims == 1 && numLeaves > 1) {
|
||||
int levelCount = 2;
|
||||
while (true) {
|
||||
if (numLeaves >= levelCount && numLeaves <= 2*levelCount) {
|
||||
int lastLevel = 2*(numLeaves - levelCount);
|
||||
assert lastLevel >= 0;
|
||||
if (lastLevel != 0) {
|
||||
// Last level is partially filled, so we must rotate the leaf FPs to match. We do this here, after loading
|
||||
// at read-time, so that we can still delta code them on disk at write:
|
||||
long[] newLeafBlockFPs = new long[numLeaves];
|
||||
System.arraycopy(leafBlockFPs, lastLevel, newLeafBlockFPs, 0, leafBlockFPs.length - lastLevel);
|
||||
System.arraycopy(leafBlockFPs, 0, newLeafBlockFPs, leafBlockFPs.length - lastLevel, lastLevel);
|
||||
leafBlockFPs = newLeafBlockFPs;
|
||||
}
|
||||
break;
|
||||
}
|
||||
|
||||
levelCount *= 2;
|
||||
}
|
||||
} else {
|
||||
splitDim = splitPackedValues[address++] & 0xff;
|
||||
}
|
||||
|
||||
assert splitDim < numDims;
|
||||
|
||||
byte[] splitPackedValue = new byte[packedBytesLength];
|
||||
|
||||
// Recurse on left sub-tree:
|
||||
System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, packedBytesLength);
|
||||
System.arraycopy(splitPackedValues, address, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
|
||||
verify(state,
|
||||
2*nodeID,
|
||||
cellMinPacked, splitPackedValue);
|
||||
|
||||
// Recurse on right sub-tree:
|
||||
System.arraycopy(cellMinPacked, 0, splitPackedValue, 0, packedBytesLength);
|
||||
System.arraycopy(splitPackedValues, address, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
|
||||
verify(state,
|
||||
2*nodeID+1,
|
||||
splitPackedValue, cellMaxPacked);
|
||||
this.leafBlockFPs = leafBlockFPs;
|
||||
packedIndex = null;
|
||||
}
|
||||
|
||||
this.in = in;
|
||||
}
|
||||
|
||||
long getMinLeafBlockFP() {
|
||||
if (packedIndex != null) {
|
||||
return new ByteArrayDataInput(packedIndex).readVLong();
|
||||
} else {
|
||||
long minFP = Long.MAX_VALUE;
|
||||
for(long fp : leafBlockFPs) {
|
||||
minFP = Math.min(minFP, fp);
|
||||
}
|
||||
return minFP;
|
||||
}
|
||||
}
|
||||
|
||||
/** Used to walk the in-heap index
|
||||
*
|
||||
* @lucene.internal */
|
||||
public abstract class IndexTree implements Cloneable {
|
||||
protected int nodeID;
|
||||
// level is 1-based so that we can do level-1 w/o checking each time:
|
||||
protected int level;
|
||||
protected int splitDim;
|
||||
protected final byte[][] splitPackedValueStack;
|
||||
|
||||
protected IndexTree() {
|
||||
int treeDepth = getTreeDepth();
|
||||
splitPackedValueStack = new byte[treeDepth+1][];
|
||||
nodeID = 1;
|
||||
level = 1;
|
||||
splitPackedValueStack[level] = new byte[packedBytesLength];
|
||||
}
|
||||
|
||||
public void pushLeft() {
|
||||
nodeID *= 2;
|
||||
level++;
|
||||
if (splitPackedValueStack[level] == null) {
|
||||
splitPackedValueStack[level] = new byte[packedBytesLength];
|
||||
}
|
||||
}
|
||||
|
||||
/** Clone, but you are not allowed to pop up past the point where the clone happened. */
|
||||
public abstract IndexTree clone();
|
||||
|
||||
public void pushRight() {
|
||||
nodeID = nodeID * 2 + 1;
|
||||
level++;
|
||||
if (splitPackedValueStack[level] == null) {
|
||||
splitPackedValueStack[level] = new byte[packedBytesLength];
|
||||
}
|
||||
}
|
||||
|
||||
public void pop() {
|
||||
nodeID /= 2;
|
||||
level--;
|
||||
splitDim = -1;
|
||||
//System.out.println(" pop nodeID=" + nodeID);
|
||||
}
|
||||
|
||||
public boolean isLeafNode() {
|
||||
return nodeID >= leafNodeOffset;
|
||||
}
|
||||
|
||||
public boolean nodeExists() {
|
||||
return nodeID - leafNodeOffset < leafNodeOffset;
|
||||
}
|
||||
|
||||
public int getNodeID() {
|
||||
return nodeID;
|
||||
}
|
||||
|
||||
public byte[] getSplitPackedValue() {
|
||||
assert isLeafNode() == false;
|
||||
assert splitPackedValueStack[level] != null: "level=" + level;
|
||||
return splitPackedValueStack[level];
|
||||
}
|
||||
|
||||
/** Only valid after pushLeft or pushRight, not pop! */
|
||||
public int getSplitDim() {
|
||||
assert isLeafNode() == false;
|
||||
return splitDim;
|
||||
}
|
||||
|
||||
/** Only valid after pushLeft or pushRight, not pop! */
|
||||
public abstract BytesRef getSplitDimValue();
|
||||
|
||||
/** Only valid after pushLeft or pushRight, not pop! */
|
||||
public abstract long getLeafBlockFP();
|
||||
}
|
||||
|
||||
/** Reads the original simple yet heap-heavy index format */
|
||||
private final class LegacyIndexTree extends IndexTree {
|
||||
|
||||
private long leafBlockFP;
|
||||
private final byte[] splitDimValue = new byte[bytesPerDim];
|
||||
private final BytesRef scratch = new BytesRef();
|
||||
|
||||
public LegacyIndexTree() {
|
||||
setNodeData();
|
||||
scratch.bytes = splitDimValue;
|
||||
scratch.length = bytesPerDim;
|
||||
}
|
||||
|
||||
@Override
|
||||
public LegacyIndexTree clone() {
|
||||
LegacyIndexTree index = new LegacyIndexTree();
|
||||
index.nodeID = nodeID;
|
||||
index.level = level;
|
||||
index.splitDim = splitDim;
|
||||
index.leafBlockFP = leafBlockFP;
|
||||
index.splitPackedValueStack[index.level] = splitPackedValueStack[index.level].clone();
|
||||
|
||||
return index;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void pushLeft() {
|
||||
super.pushLeft();
|
||||
setNodeData();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void pushRight() {
|
||||
super.pushRight();
|
||||
setNodeData();
|
||||
}
|
||||
|
||||
private void setNodeData() {
|
||||
if (isLeafNode()) {
|
||||
leafBlockFP = leafBlockFPs[nodeID - leafNodeOffset];
|
||||
splitDim = -1;
|
||||
} else {
|
||||
leafBlockFP = -1;
|
||||
int address = nodeID * bytesPerIndexEntry;
|
||||
if (numDims == 1) {
|
||||
splitDim = 0;
|
||||
if (version < BKDWriter.VERSION_IMPLICIT_SPLIT_DIM_1D) {
|
||||
// skip over wastefully encoded 0 splitDim:
|
||||
assert splitPackedValues[address] == 0;
|
||||
address++;
|
||||
}
|
||||
} else {
|
||||
splitDim = splitPackedValues[address++] & 0xff;
|
||||
}
|
||||
System.arraycopy(splitPackedValues, address, splitDimValue, 0, bytesPerDim);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLeafBlockFP() {
|
||||
assert isLeafNode();
|
||||
return leafBlockFP;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getSplitDimValue() {
|
||||
assert isLeafNode() == false;
|
||||
return scratch;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void pop() {
|
||||
super.pop();
|
||||
leafBlockFP = -1;
|
||||
}
|
||||
}
|
||||
|
||||
/** Reads the new packed byte[] index format which can be up to ~63% smaller than the legacy index format on 20M NYC taxis tests. This
|
||||
* format takes advantage of the limited access pattern to the BKD tree at search time, i.e. starting at the root node and recursing
|
||||
* downwards one child at a time. */
|
||||
private final class PackedIndexTree extends IndexTree {
|
||||
// used to read the packed byte[]
|
||||
private final ByteArrayDataInput in;
|
||||
// holds the minimum (left most) leaf block file pointer for each level we've recursed to:
|
||||
private final long[] leafBlockFPStack;
|
||||
// holds the address, in the packed byte[] index, of the left-node of each level:
|
||||
private final int[] leftNodePositions;
|
||||
// holds the address, in the packed byte[] index, of the right-node of each level:
|
||||
private final int[] rightNodePositions;
|
||||
// holds the splitDim for each level:
|
||||
private final int[] splitDims;
|
||||
// true if the per-dim delta we read for the node at this level is a negative offset vs. the last split on this dim; this is a packed
|
||||
// 2D array, i.e. to access array[level][dim] you read from negativeDeltas[level*numDims+dim]. this will be true if the last time we
|
||||
// split on this dimension, we next pushed to the left sub-tree:
|
||||
private final boolean[] negativeDeltas;
|
||||
// holds the packed per-level split values; the intersect method uses this to save the cell min/max as it recurses:
|
||||
private final byte[][] splitValuesStack;
|
||||
// scratch value to return from getPackedValue:
|
||||
private final BytesRef scratch;
|
||||
|
||||
public PackedIndexTree() {
|
||||
int treeDepth = getTreeDepth();
|
||||
leafBlockFPStack = new long[treeDepth+1];
|
||||
leftNodePositions = new int[treeDepth+1];
|
||||
rightNodePositions = new int[treeDepth+1];
|
||||
splitValuesStack = new byte[treeDepth+1][];
|
||||
splitDims = new int[treeDepth+1];
|
||||
negativeDeltas = new boolean[numDims*(treeDepth+1)];
|
||||
|
||||
in = new ByteArrayDataInput(packedIndex);
|
||||
splitValuesStack[0] = new byte[packedBytesLength];
|
||||
readNodeData(false);
|
||||
scratch = new BytesRef();
|
||||
scratch.length = bytesPerDim;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PackedIndexTree clone() {
|
||||
PackedIndexTree index = new PackedIndexTree();
|
||||
index.nodeID = nodeID;
|
||||
index.level = level;
|
||||
index.splitDim = splitDim;
|
||||
System.arraycopy(negativeDeltas, level*numDims, index.negativeDeltas, level*numDims, numDims);
|
||||
index.leafBlockFPStack[level] = leafBlockFPStack[level];
|
||||
index.leftNodePositions[level] = leftNodePositions[level];
|
||||
index.rightNodePositions[level] = rightNodePositions[level];
|
||||
index.splitValuesStack[index.level] = splitValuesStack[index.level].clone();
|
||||
System.arraycopy(negativeDeltas, level*numDims, index.negativeDeltas, level*numDims, numDims);
|
||||
index.splitDims[level] = splitDims[level];
|
||||
return index;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void pushLeft() {
|
||||
int nodePosition = leftNodePositions[level];
|
||||
super.pushLeft();
|
||||
System.arraycopy(negativeDeltas, (level-1)*numDims, negativeDeltas, level*numDims, numDims);
|
||||
assert splitDim != -1;
|
||||
negativeDeltas[level*numDims+splitDim] = true;
|
||||
in.setPosition(nodePosition);
|
||||
readNodeData(true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void pushRight() {
|
||||
int nodePosition = rightNodePositions[level];
|
||||
super.pushRight();
|
||||
System.arraycopy(negativeDeltas, (level-1)*numDims, negativeDeltas, level*numDims, numDims);
|
||||
assert splitDim != -1;
|
||||
negativeDeltas[level*numDims+splitDim] = false;
|
||||
in.setPosition(nodePosition);
|
||||
readNodeData(false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void pop() {
|
||||
super.pop();
|
||||
splitDim = splitDims[level];
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLeafBlockFP() {
|
||||
assert isLeafNode(): "nodeID=" + nodeID + " is not a leaf";
|
||||
return leafBlockFPStack[level];
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getSplitDimValue() {
|
||||
assert isLeafNode() == false;
|
||||
scratch.bytes = splitValuesStack[level];
|
||||
scratch.offset = splitDim * bytesPerDim;
|
||||
return scratch;
|
||||
}
|
||||
|
||||
private void readNodeData(boolean isLeft) {
|
||||
|
||||
leafBlockFPStack[level] = leafBlockFPStack[level-1];
|
||||
|
||||
// read leaf block FP delta
|
||||
if (isLeft == false) {
|
||||
leafBlockFPStack[level] += in.readVLong();
|
||||
}
|
||||
|
||||
if (isLeafNode()) {
|
||||
splitDim = -1;
|
||||
} else {
|
||||
|
||||
// read split dim, prefix, firstDiffByteDelta encoded as int:
|
||||
int code = in.readVInt();
|
||||
splitDim = code % numDims;
|
||||
splitDims[level] = splitDim;
|
||||
code /= numDims;
|
||||
int prefix = code % (1+bytesPerDim);
|
||||
int suffix = bytesPerDim - prefix;
|
||||
|
||||
if (splitValuesStack[level] == null) {
|
||||
splitValuesStack[level] = new byte[packedBytesLength];
|
||||
}
|
||||
System.arraycopy(splitValuesStack[level-1], 0, splitValuesStack[level], 0, packedBytesLength);
|
||||
if (suffix > 0) {
|
||||
int firstDiffByteDelta = code / (1+bytesPerDim);
|
||||
if (negativeDeltas[level*numDims + splitDim]) {
|
||||
firstDiffByteDelta = -firstDiffByteDelta;
|
||||
}
|
||||
int oldByte = splitValuesStack[level][splitDim*bytesPerDim+prefix] & 0xFF;
|
||||
splitValuesStack[level][splitDim*bytesPerDim+prefix] = (byte) (oldByte + firstDiffByteDelta);
|
||||
in.readBytes(splitValuesStack[level], splitDim*bytesPerDim+prefix+1, suffix-1);
|
||||
} else {
|
||||
// our split value is == last split value in this dim, which can happen when there are many duplicate values
|
||||
}
|
||||
|
||||
int leftNumBytes;
|
||||
if (nodeID * 2 < leafNodeOffset) {
|
||||
leftNumBytes = in.readVInt();
|
||||
} else {
|
||||
leftNumBytes = 0;
|
||||
}
|
||||
|
||||
leftNodePositions[level] = in.getPosition();
|
||||
rightNodePositions[level] = leftNodePositions[level] + leftNumBytes;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private int getTreeDepth() {
|
||||
// First +1 because all the non-leave nodes makes another power
|
||||
// of 2; e.g. to have a fully balanced tree with 4 leaves you
|
||||
// need a depth=3 tree:
|
||||
|
||||
// Second +1 because MathUtil.log computes floor of the logarithm; e.g.
|
||||
// with 5 leaves you need a depth=4 tree:
|
||||
return MathUtil.log(numLeaves, 2) + 2;
|
||||
}
|
||||
|
||||
/** Used to track all state for a single call to {@link #intersect}. */
|
||||
|
@ -285,57 +467,73 @@ public class BKDReader extends PointValues implements Accountable {
|
|||
final int[] commonPrefixLengths;
|
||||
|
||||
final IntersectVisitor visitor;
|
||||
public final IndexTree index;
|
||||
|
||||
public IntersectState(IndexInput in, int numDims,
|
||||
int packedBytesLength,
|
||||
int maxPointsInLeafNode,
|
||||
IntersectVisitor visitor) {
|
||||
IntersectVisitor visitor,
|
||||
IndexTree indexVisitor) {
|
||||
this.in = in;
|
||||
this.visitor = visitor;
|
||||
this.commonPrefixLengths = new int[numDims];
|
||||
this.scratchDocIDs = new int[maxPointsInLeafNode];
|
||||
this.scratchPackedValue = new byte[packedBytesLength];
|
||||
this.index = indexVisitor;
|
||||
}
|
||||
}
|
||||
|
||||
public void intersect(IntersectVisitor visitor) throws IOException {
|
||||
intersect(getIntersectState(visitor), 1, minPackedValue, maxPackedValue);
|
||||
intersect(getIntersectState(visitor), minPackedValue, maxPackedValue);
|
||||
}
|
||||
|
||||
/** Fast path: this is called when the query box fully encompasses all cells under this node. */
|
||||
private void addAll(IntersectState state, int nodeID) throws IOException {
|
||||
private void addAll(IntersectState state) throws IOException {
|
||||
//System.out.println("R: addAll nodeID=" + nodeID);
|
||||
|
||||
if (nodeID >= leafNodeOffset) {
|
||||
if (state.index.isLeafNode()) {
|
||||
//System.out.println("ADDALL");
|
||||
visitDocIDs(state.in, leafBlockFPs[nodeID-leafNodeOffset], state.visitor);
|
||||
if (state.index.nodeExists()) {
|
||||
visitDocIDs(state.in, state.index.getLeafBlockFP(), state.visitor);
|
||||
}
|
||||
// TODO: we can assert that the first value here in fact matches what the index claimed?
|
||||
} else {
|
||||
addAll(state, 2*nodeID);
|
||||
addAll(state, 2*nodeID+1);
|
||||
state.index.pushLeft();
|
||||
addAll(state);
|
||||
state.index.pop();
|
||||
|
||||
state.index.pushRight();
|
||||
addAll(state);
|
||||
state.index.pop();
|
||||
}
|
||||
}
|
||||
|
||||
/** Create a new {@link IntersectState} */
|
||||
public IntersectState getIntersectState(IntersectVisitor visitor) {
|
||||
IndexTree index;
|
||||
if (packedIndex != null) {
|
||||
index = new PackedIndexTree();
|
||||
} else {
|
||||
index = new LegacyIndexTree();
|
||||
}
|
||||
return new IntersectState(in.clone(), numDims,
|
||||
packedBytesLength,
|
||||
maxPointsInLeafNode,
|
||||
visitor);
|
||||
visitor,
|
||||
index);
|
||||
}
|
||||
|
||||
/** Visits all docIDs and packed values in a single leaf block */
|
||||
public void visitLeafBlockValues(int nodeID, IntersectState state) throws IOException {
|
||||
int leafID = nodeID - leafNodeOffset;
|
||||
public void visitLeafBlockValues(IndexTree index, IntersectState state) throws IOException {
|
||||
|
||||
// Leaf node; scan and filter all points in this block:
|
||||
int count = readDocIDs(state.in, leafBlockFPs[leafID], state.scratchDocIDs);
|
||||
int count = readDocIDs(state.in, index.getLeafBlockFP(), state.scratchDocIDs);
|
||||
|
||||
// Again, this time reading values and checking with the visitor
|
||||
visitDocValues(state.commonPrefixLengths, state.scratchPackedValue, state.in, state.scratchDocIDs, count, state.visitor);
|
||||
}
|
||||
|
||||
protected void visitDocIDs(IndexInput in, long blockFP, IntersectVisitor visitor) throws IOException {
|
||||
private void visitDocIDs(IndexInput in, long blockFP, IntersectVisitor visitor) throws IOException {
|
||||
// Leaf node
|
||||
in.seek(blockFP);
|
||||
|
||||
|
@ -350,7 +548,7 @@ public class BKDReader extends PointValues implements Accountable {
|
|||
}
|
||||
}
|
||||
|
||||
protected int readDocIDs(IndexInput in, long blockFP, int[] docIDs) throws IOException {
|
||||
int readDocIDs(IndexInput in, long blockFP, int[] docIDs) throws IOException {
|
||||
in.seek(blockFP);
|
||||
|
||||
// How many points are stored in this leaf cell:
|
||||
|
@ -365,7 +563,7 @@ public class BKDReader extends PointValues implements Accountable {
|
|||
return count;
|
||||
}
|
||||
|
||||
protected void visitDocValues(int[] commonPrefixLengths, byte[] scratchPackedValue, IndexInput in, int[] docIDs, int count, IntersectVisitor visitor) throws IOException {
|
||||
void visitDocValues(int[] commonPrefixLengths, byte[] scratchPackedValue, IndexInput in, int[] docIDs, int count, IntersectVisitor visitor) throws IOException {
|
||||
visitor.grow(count);
|
||||
|
||||
readCommonPrefixes(commonPrefixLengths, scratchPackedValue, in);
|
||||
|
@ -434,13 +632,10 @@ public class BKDReader extends PointValues implements Accountable {
|
|||
}
|
||||
}
|
||||
|
||||
private void intersect(IntersectState state,
|
||||
int nodeID,
|
||||
byte[] cellMinPacked, byte[] cellMaxPacked)
|
||||
throws IOException {
|
||||
private void intersect(IntersectState state, byte[] cellMinPacked, byte[] cellMaxPacked) throws IOException {
|
||||
|
||||
/*
|
||||
System.out.println("\nR: intersect nodeID=" + nodeID);
|
||||
System.out.println("\nR: intersect nodeID=" + state.index.getNodeID());
|
||||
for(int dim=0;dim<numDims;dim++) {
|
||||
System.out.println(" dim=" + dim + "\n cellMin=" + new BytesRef(cellMinPacked, dim*bytesPerDim, bytesPerDim) + "\n cellMax=" + new BytesRef(cellMaxPacked, dim*bytesPerDim, bytesPerDim));
|
||||
}
|
||||
|
@ -450,24 +645,18 @@ public class BKDReader extends PointValues implements Accountable {
|
|||
|
||||
if (r == Relation.CELL_OUTSIDE_QUERY) {
|
||||
// This cell is fully outside of the query shape: stop recursing
|
||||
return;
|
||||
} else if (r == Relation.CELL_INSIDE_QUERY) {
|
||||
// This cell is fully inside of the query shape: recursively add all points in this cell without filtering
|
||||
addAll(state, nodeID);
|
||||
return;
|
||||
} else {
|
||||
// The cell crosses the shape boundary, or the cell fully contains the query, so we fall through and do full filtering
|
||||
}
|
||||
|
||||
if (nodeID >= leafNodeOffset) {
|
||||
addAll(state);
|
||||
// The cell crosses the shape boundary, or the cell fully contains the query, so we fall through and do full filtering:
|
||||
} else if (state.index.isLeafNode()) {
|
||||
|
||||
// TODO: we can assert that the first value here in fact matches what the index claimed?
|
||||
|
||||
int leafID = nodeID - leafNodeOffset;
|
||||
|
||||
// In the unbalanced case it's possible the left most node only has one child:
|
||||
if (leafID < leafBlockFPs.length) {
|
||||
if (state.index.nodeExists()) {
|
||||
// Leaf node; scan and filter all points in this block:
|
||||
int count = readDocIDs(state.in, leafBlockFPs[leafID], state.scratchDocIDs);
|
||||
int count = readDocIDs(state.in, state.index.getLeafBlockFP(), state.scratchDocIDs);
|
||||
|
||||
// Again, this time reading values and checking with the visitor
|
||||
visitDocValues(state.commonPrefixLengths, state.scratchPackedValue, state.in, state.scratchDocIDs, count, state.visitor);
|
||||
|
@ -476,65 +665,45 @@ public class BKDReader extends PointValues implements Accountable {
|
|||
} else {
|
||||
|
||||
// Non-leaf node: recurse on the split left and right nodes
|
||||
|
||||
int address = nodeID * bytesPerIndexEntry;
|
||||
int splitDim;
|
||||
if (numDims == 1) {
|
||||
splitDim = 0;
|
||||
if (version < BKDWriter.VERSION_IMPLICIT_SPLIT_DIM_1D) {
|
||||
// skip over wastefully encoded 0 splitDim:
|
||||
assert splitPackedValues[address] == 0;
|
||||
address++;
|
||||
}
|
||||
} else {
|
||||
splitDim = splitPackedValues[address++] & 0xff;
|
||||
}
|
||||
|
||||
int splitDim = state.index.getSplitDim();
|
||||
assert splitDim >= 0: "splitDim=" + splitDim;
|
||||
assert splitDim < numDims;
|
||||
|
||||
// TODO: can we alloc & reuse this up front?
|
||||
byte[] splitPackedValue = state.index.getSplitPackedValue();
|
||||
BytesRef splitDimValue = state.index.getSplitDimValue();
|
||||
assert splitDimValue.length == bytesPerDim;
|
||||
//System.out.println(" splitDimValue=" + splitDimValue + " splitDim=" + splitDim);
|
||||
|
||||
byte[] splitPackedValue = new byte[packedBytesLength];
|
||||
// make sure cellMin <= splitValue <= cellMax:
|
||||
assert StringHelper.compare(bytesPerDim, cellMinPacked, splitDim*bytesPerDim, splitDimValue.bytes, splitDimValue.offset) <= 0: "bytesPerDim=" + bytesPerDim + " splitDim=" + splitDim + " numDims=" + numDims;
|
||||
assert StringHelper.compare(bytesPerDim, cellMaxPacked, splitDim*bytesPerDim, splitDimValue.bytes, splitDimValue.offset) >= 0: "bytesPerDim=" + bytesPerDim + " splitDim=" + splitDim + " numDims=" + numDims;
|
||||
|
||||
// Recurse on left sub-tree:
|
||||
System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, packedBytesLength);
|
||||
System.arraycopy(splitPackedValues, address, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
|
||||
intersect(state,
|
||||
2*nodeID,
|
||||
cellMinPacked, splitPackedValue);
|
||||
System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
|
||||
state.index.pushLeft();
|
||||
intersect(state, cellMinPacked, splitPackedValue);
|
||||
state.index.pop();
|
||||
|
||||
// Restore the split dim value since it may have been overwritten while recursing:
|
||||
System.arraycopy(splitPackedValue, splitDim*bytesPerDim, splitDimValue.bytes, splitDimValue.offset, bytesPerDim);
|
||||
|
||||
// Recurse on right sub-tree:
|
||||
System.arraycopy(cellMinPacked, 0, splitPackedValue, 0, packedBytesLength);
|
||||
System.arraycopy(splitPackedValues, address, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
|
||||
intersect(state,
|
||||
2*nodeID+1,
|
||||
splitPackedValue, cellMaxPacked);
|
||||
System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
|
||||
state.index.pushRight();
|
||||
intersect(state, splitPackedValue, cellMaxPacked);
|
||||
state.index.pop();
|
||||
}
|
||||
}
|
||||
|
||||
/** Copies the split value for this node into the provided byte array */
|
||||
public void copySplitValue(int nodeID, byte[] splitPackedValue) {
|
||||
int address = nodeID * bytesPerIndexEntry;
|
||||
int splitDim;
|
||||
if (numDims == 1) {
|
||||
splitDim = 0;
|
||||
if (version < BKDWriter.VERSION_IMPLICIT_SPLIT_DIM_1D) {
|
||||
// skip over wastefully encoded 0 splitDim:
|
||||
assert splitPackedValues[address] == 0;
|
||||
address++;
|
||||
}
|
||||
} else {
|
||||
splitDim = splitPackedValues[address++] & 0xff;
|
||||
}
|
||||
|
||||
assert splitDim < numDims;
|
||||
System.arraycopy(splitPackedValues, address, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return RamUsageEstimator.sizeOf(splitPackedValues) +
|
||||
RamUsageEstimator.sizeOf(leafBlockFPs);
|
||||
if (packedIndex != null) {
|
||||
return packedIndex.length;
|
||||
} else {
|
||||
return RamUsageEstimator.sizeOf(splitPackedValues) + RamUsageEstimator.sizeOf(leafBlockFPs);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -33,6 +33,7 @@ import org.apache.lucene.store.ChecksumIndexInput;
|
|||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.store.RAMOutputStream;
|
||||
import org.apache.lucene.store.TrackingDirectoryWrapper;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
@ -83,7 +84,8 @@ public class BKDWriter implements Closeable {
|
|||
public static final int VERSION_COMPRESSED_DOC_IDS = 1;
|
||||
public static final int VERSION_COMPRESSED_VALUES = 2;
|
||||
public static final int VERSION_IMPLICIT_SPLIT_DIM_1D = 3;
|
||||
public static final int VERSION_CURRENT = VERSION_IMPLICIT_SPLIT_DIM_1D;
|
||||
public static final int VERSION_PACKED_INDEX = 4;
|
||||
public static final int VERSION_CURRENT = VERSION_PACKED_INDEX;
|
||||
|
||||
/** How many bytes each docs takes in the fixed-width offline format */
|
||||
private final int bytesPerDoc;
|
||||
|
@ -325,15 +327,10 @@ public class BKDWriter implements Closeable {
|
|||
bkd.numDims,
|
||||
bkd.packedBytesLength,
|
||||
bkd.maxPointsInLeafNode,
|
||||
null,
|
||||
null);
|
||||
this.docMap = docMap;
|
||||
long minFP = Long.MAX_VALUE;
|
||||
//System.out.println("MR.init " + this + " bkdreader=" + bkd + " leafBlockFPs.length=" + bkd.leafBlockFPs.length);
|
||||
for(long fp : bkd.leafBlockFPs) {
|
||||
minFP = Math.min(minFP, fp);
|
||||
//System.out.println(" leaf fp=" + fp);
|
||||
}
|
||||
state.in.seek(minFP);
|
||||
state.in.seek(bkd.getMinLeafBlockFP());
|
||||
this.packedValues = new byte[bkd.maxPointsInLeafNode * bkd.packedBytesLength];
|
||||
}
|
||||
|
||||
|
@ -341,7 +338,7 @@ public class BKDWriter implements Closeable {
|
|||
//System.out.println("MR.next this=" + this);
|
||||
while (true) {
|
||||
if (docBlockUpto == docsInBlock) {
|
||||
if (blockID == bkd.leafBlockFPs.length) {
|
||||
if (blockID == bkd.leafNodeOffset) {
|
||||
//System.out.println(" done!");
|
||||
return false;
|
||||
}
|
||||
|
@ -489,7 +486,6 @@ public class BKDWriter implements Closeable {
|
|||
return indexFP;
|
||||
}
|
||||
|
||||
|
||||
/* In the 1D case, we can simply sort points in ascending order and use the
|
||||
* same writing logic as we use at merge time. */
|
||||
private long writeField1Dim(IndexOutput out, String fieldName, MutablePointValues reader) throws IOException {
|
||||
|
@ -648,6 +644,7 @@ public class BKDWriter implements Closeable {
|
|||
}
|
||||
|
||||
private void writeLeafBlock() throws IOException {
|
||||
//System.out.println("writeLeafBlock pos=" + out.getFilePointer());
|
||||
assert leafCount != 0;
|
||||
if (valueCount == 0) {
|
||||
System.arraycopy(leafValues, 0, minPackedValue, 0, packedBytesLength);
|
||||
|
@ -811,6 +808,24 @@ public class BKDWriter implements Closeable {
|
|||
}.sort(0, pointCount);
|
||||
}
|
||||
|
||||
// useful for debugging:
|
||||
/*
|
||||
private void printPathSlice(String desc, PathSlice slice, int dim) throws IOException {
|
||||
System.out.println(" " + desc + " dim=" + dim + " count=" + slice.count + ":");
|
||||
try(PointReader r = slice.writer.getReader(slice.start, slice.count)) {
|
||||
int count = 0;
|
||||
while (r.next()) {
|
||||
byte[] v = r.packedValue();
|
||||
System.out.println(" " + count + ": " + new BytesRef(v, dim*bytesPerDim, bytesPerDim));
|
||||
count++;
|
||||
if (count == slice.count) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
*/
|
||||
|
||||
private PointWriter sort(int dim) throws IOException {
|
||||
assert dim >= 0 && dim < numDims;
|
||||
|
||||
|
@ -1019,46 +1034,238 @@ public class BKDWriter implements Closeable {
|
|||
return indexFP;
|
||||
}
|
||||
|
||||
/** Subclass can change how it writes the index. */
|
||||
protected void writeIndex(IndexOutput out, long[] leafBlockFPs, byte[] splitPackedValues) throws IOException {
|
||||
/** Packs the two arrays, representing a balanced binary tree, into a compact byte[] structure. */
|
||||
private byte[] packIndex(long[] leafBlockFPs, byte[] splitPackedValues) throws IOException {
|
||||
|
||||
int numLeaves = leafBlockFPs.length;
|
||||
|
||||
// Possibly rotate the leaf block FPs, if the index not fully balanced binary tree (only happens
|
||||
// if it was created by OneDimensionBKDWriter). In this case the leaf nodes may straddle the two bottom
|
||||
// levels of the binary tree:
|
||||
if (numDims == 1 && numLeaves > 1) {
|
||||
int levelCount = 2;
|
||||
while (true) {
|
||||
if (numLeaves >= levelCount && numLeaves <= 2*levelCount) {
|
||||
int lastLevel = 2*(numLeaves - levelCount);
|
||||
assert lastLevel >= 0;
|
||||
if (lastLevel != 0) {
|
||||
// Last level is partially filled, so we must rotate the leaf FPs to match. We do this here, after loading
|
||||
// at read-time, so that we can still delta code them on disk at write:
|
||||
long[] newLeafBlockFPs = new long[numLeaves];
|
||||
System.arraycopy(leafBlockFPs, lastLevel, newLeafBlockFPs, 0, leafBlockFPs.length - lastLevel);
|
||||
System.arraycopy(leafBlockFPs, 0, newLeafBlockFPs, leafBlockFPs.length - lastLevel, lastLevel);
|
||||
leafBlockFPs = newLeafBlockFPs;
|
||||
}
|
||||
break;
|
||||
}
|
||||
|
||||
levelCount *= 2;
|
||||
}
|
||||
}
|
||||
|
||||
/** Reused while packing the index */
|
||||
RAMOutputStream writeBuffer = new RAMOutputStream();
|
||||
|
||||
// This is the "file" we append the byte[] to:
|
||||
List<byte[]> blocks = new ArrayList<>();
|
||||
byte[] lastSplitValues = new byte[bytesPerDim * numDims];
|
||||
//System.out.println("\npack index");
|
||||
int totalSize = recursePackIndex(writeBuffer, leafBlockFPs, splitPackedValues, 0l, blocks, 1, lastSplitValues, new boolean[numDims], false);
|
||||
|
||||
// Compact the byte[] blocks into single byte index:
|
||||
byte[] index = new byte[totalSize];
|
||||
int upto = 0;
|
||||
for(byte[] block : blocks) {
|
||||
System.arraycopy(block, 0, index, upto, block.length);
|
||||
upto += block.length;
|
||||
}
|
||||
assert upto == totalSize;
|
||||
|
||||
return index;
|
||||
}
|
||||
|
||||
/** Appends the current contents of writeBuffer as another block on the growing in-memory file */
|
||||
private int appendBlock(RAMOutputStream writeBuffer, List<byte[]> blocks) throws IOException {
|
||||
int pos = Math.toIntExact(writeBuffer.getFilePointer());
|
||||
byte[] bytes = new byte[pos];
|
||||
writeBuffer.writeTo(bytes, 0);
|
||||
writeBuffer.reset();
|
||||
blocks.add(bytes);
|
||||
return pos;
|
||||
}
|
||||
|
||||
/**
|
||||
* lastSplitValues is per-dimension split value previously seen; we use this to prefix-code the split byte[] on each inner node
|
||||
*/
|
||||
private int recursePackIndex(RAMOutputStream writeBuffer, long[] leafBlockFPs, byte[] splitPackedValues, long minBlockFP, List<byte[]> blocks,
|
||||
int nodeID, byte[] lastSplitValues, boolean[] negativeDeltas, boolean isLeft) throws IOException {
|
||||
if (nodeID >= leafBlockFPs.length) {
|
||||
int leafID = nodeID - leafBlockFPs.length;
|
||||
//System.out.println("recursePack leaf nodeID=" + nodeID);
|
||||
|
||||
// In the unbalanced case it's possible the left most node only has one child:
|
||||
if (leafID < leafBlockFPs.length) {
|
||||
long delta = leafBlockFPs[leafID] - minBlockFP;
|
||||
if (isLeft) {
|
||||
assert delta == 0;
|
||||
return 0;
|
||||
} else {
|
||||
assert nodeID == 1 || delta > 0: "nodeID=" + nodeID;
|
||||
writeBuffer.writeVLong(delta);
|
||||
return appendBlock(writeBuffer, blocks);
|
||||
}
|
||||
} else {
|
||||
return 0;
|
||||
}
|
||||
} else {
|
||||
long leftBlockFP;
|
||||
if (isLeft == false) {
|
||||
leftBlockFP = getLeftMostLeafBlockFP(leafBlockFPs, nodeID);
|
||||
long delta = leftBlockFP - minBlockFP;
|
||||
assert nodeID == 1 || delta > 0;
|
||||
writeBuffer.writeVLong(delta);
|
||||
} else {
|
||||
// The left tree's left most leaf block FP is always the minimal FP:
|
||||
leftBlockFP = minBlockFP;
|
||||
}
|
||||
|
||||
int address = nodeID * (1+bytesPerDim);
|
||||
int splitDim = splitPackedValues[address++] & 0xff;
|
||||
|
||||
//System.out.println("recursePack inner nodeID=" + nodeID + " splitDim=" + splitDim + " splitValue=" + new BytesRef(splitPackedValues, address, bytesPerDim));
|
||||
|
||||
// find common prefix with last split value in this dim:
|
||||
int prefix = 0;
|
||||
for(;prefix<bytesPerDim;prefix++) {
|
||||
if (splitPackedValues[address+prefix] != lastSplitValues[splitDim * bytesPerDim + prefix]) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
//System.out.println("writeNodeData nodeID=" + nodeID + " splitDim=" + splitDim + " numDims=" + numDims + " bytesPerDim=" + bytesPerDim + " prefix=" + prefix);
|
||||
|
||||
int firstDiffByteDelta;
|
||||
if (prefix < bytesPerDim) {
|
||||
//System.out.println(" delta byte cur=" + Integer.toHexString(splitPackedValues[address+prefix]&0xFF) + " prev=" + Integer.toHexString(lastSplitValues[splitDim * bytesPerDim + prefix]&0xFF) + " negated?=" + negativeDeltas[splitDim]);
|
||||
firstDiffByteDelta = (splitPackedValues[address+prefix]&0xFF) - (lastSplitValues[splitDim * bytesPerDim + prefix]&0xFF);
|
||||
if (negativeDeltas[splitDim]) {
|
||||
firstDiffByteDelta = -firstDiffByteDelta;
|
||||
}
|
||||
//System.out.println(" delta=" + firstDiffByteDelta);
|
||||
assert firstDiffByteDelta > 0;
|
||||
} else {
|
||||
firstDiffByteDelta = 0;
|
||||
}
|
||||
|
||||
// pack the prefix, splitDim and delta first diff byte into a single vInt:
|
||||
int code = (firstDiffByteDelta * (1+bytesPerDim) + prefix) * numDims + splitDim;
|
||||
|
||||
//System.out.println(" code=" + code);
|
||||
//System.out.println(" splitValue=" + new BytesRef(splitPackedValues, address, bytesPerDim));
|
||||
|
||||
writeBuffer.writeVInt(code);
|
||||
|
||||
// write the split value, prefix coded vs. our parent's split value:
|
||||
int suffix = bytesPerDim - prefix;
|
||||
byte[] savSplitValue = new byte[suffix];
|
||||
if (suffix > 1) {
|
||||
writeBuffer.writeBytes(splitPackedValues, address+prefix+1, suffix-1);
|
||||
}
|
||||
|
||||
byte[] cmp = lastSplitValues.clone();
|
||||
|
||||
System.arraycopy(lastSplitValues, splitDim * bytesPerDim + prefix, savSplitValue, 0, suffix);
|
||||
|
||||
// copy our split value into lastSplitValues for our children to prefix-code against
|
||||
System.arraycopy(splitPackedValues, address+prefix, lastSplitValues, splitDim * bytesPerDim + prefix, suffix);
|
||||
|
||||
int numBytes = appendBlock(writeBuffer, blocks);
|
||||
|
||||
// placeholder for left-tree numBytes; we need this so that at search time if we only need to recurse into the right sub-tree we can
|
||||
// quickly seek to its starting point
|
||||
int idxSav = blocks.size();
|
||||
blocks.add(null);
|
||||
|
||||
boolean savNegativeDelta = negativeDeltas[splitDim];
|
||||
negativeDeltas[splitDim] = true;
|
||||
|
||||
int leftNumBytes = recursePackIndex(writeBuffer, leafBlockFPs, splitPackedValues, leftBlockFP, blocks, 2*nodeID, lastSplitValues, negativeDeltas, true);
|
||||
|
||||
if (nodeID * 2 < leafBlockFPs.length) {
|
||||
writeBuffer.writeVInt(leftNumBytes);
|
||||
} else {
|
||||
assert leftNumBytes == 0: "leftNumBytes=" + leftNumBytes;
|
||||
}
|
||||
int numBytes2 = Math.toIntExact(writeBuffer.getFilePointer());
|
||||
byte[] bytes2 = new byte[numBytes2];
|
||||
writeBuffer.writeTo(bytes2, 0);
|
||||
writeBuffer.reset();
|
||||
// replace our placeholder:
|
||||
blocks.set(idxSav, bytes2);
|
||||
|
||||
negativeDeltas[splitDim] = false;
|
||||
int rightNumBytes = recursePackIndex(writeBuffer, leafBlockFPs, splitPackedValues, leftBlockFP, blocks, 2*nodeID+1, lastSplitValues, negativeDeltas, false);
|
||||
|
||||
negativeDeltas[splitDim] = savNegativeDelta;
|
||||
|
||||
// restore lastSplitValues to what caller originally passed us:
|
||||
System.arraycopy(savSplitValue, 0, lastSplitValues, splitDim * bytesPerDim + prefix, suffix);
|
||||
|
||||
assert Arrays.equals(lastSplitValues, cmp);
|
||||
|
||||
return numBytes + numBytes2 + leftNumBytes + rightNumBytes;
|
||||
}
|
||||
}
|
||||
|
||||
private long getLeftMostLeafBlockFP(long[] leafBlockFPs, int nodeID) {
|
||||
int nodeIDIn = nodeID;
|
||||
// TODO: can we do this cheaper, e.g. a closed form solution instead of while loop? Or
|
||||
// change the recursion while packing the index to return this left-most leaf block FP
|
||||
// from each recursion instead?
|
||||
//
|
||||
// Still, the overall cost here is minor: this method's cost is O(log(N)), and while writing
|
||||
// we call it O(N) times (N = number of leaf blocks)
|
||||
while (nodeID < leafBlockFPs.length) {
|
||||
nodeID *= 2;
|
||||
}
|
||||
int leafID = nodeID - leafBlockFPs.length;
|
||||
long result = leafBlockFPs[leafID];
|
||||
if (result < 0) {
|
||||
throw new AssertionError(result + " for leaf " + leafID);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
private void writeIndex(IndexOutput out, long[] leafBlockFPs, byte[] splitPackedValues) throws IOException {
|
||||
byte[] packedIndex = packIndex(leafBlockFPs, splitPackedValues);
|
||||
writeIndex(out, leafBlockFPs.length, packedIndex);
|
||||
}
|
||||
|
||||
private void writeIndex(IndexOutput out, int numLeaves, byte[] packedIndex) throws IOException {
|
||||
|
||||
CodecUtil.writeHeader(out, CODEC_NAME, VERSION_CURRENT);
|
||||
out.writeVInt(numDims);
|
||||
out.writeVInt(maxPointsInLeafNode);
|
||||
out.writeVInt(bytesPerDim);
|
||||
|
||||
assert leafBlockFPs.length > 0;
|
||||
out.writeVInt(leafBlockFPs.length);
|
||||
assert numLeaves > 0;
|
||||
out.writeVInt(numLeaves);
|
||||
out.writeBytes(minPackedValue, 0, packedBytesLength);
|
||||
out.writeBytes(maxPackedValue, 0, packedBytesLength);
|
||||
|
||||
out.writeVLong(pointCount);
|
||||
out.writeVInt(docsSeen.cardinality());
|
||||
|
||||
// NOTE: splitPackedValues[0] is unused, because nodeID is 1-based:
|
||||
if (numDims == 1) {
|
||||
// write the index, skipping the byte used to store the split dim since it is always 0
|
||||
for (int i = 1; i < splitPackedValues.length; i += 1 + bytesPerDim) {
|
||||
out.writeBytes(splitPackedValues, i, bytesPerDim);
|
||||
}
|
||||
} else {
|
||||
out.writeBytes(splitPackedValues, 0, splitPackedValues.length);
|
||||
}
|
||||
|
||||
long lastFP = 0;
|
||||
for (int i=0;i<leafBlockFPs.length;i++) {
|
||||
long delta = leafBlockFPs[i]-lastFP;
|
||||
out.writeVLong(delta);
|
||||
lastFP = leafBlockFPs[i];
|
||||
}
|
||||
out.writeVInt(packedIndex.length);
|
||||
out.writeBytes(packedIndex, 0, packedIndex.length);
|
||||
}
|
||||
|
||||
protected void writeLeafBlockDocs(IndexOutput out, int[] docIDs, int start, int count) throws IOException {
|
||||
private void writeLeafBlockDocs(IndexOutput out, int[] docIDs, int start, int count) throws IOException {
|
||||
assert count > 0: "maxPointsInLeafNode=" + maxPointsInLeafNode;
|
||||
out.writeVInt(count);
|
||||
DocIdsWriter.writeDocIds(docIDs, start, count, out);
|
||||
}
|
||||
|
||||
protected void writeLeafBlockPackedValues(IndexOutput out, int[] commonPrefixLengths, int count, int sortedDim, IntFunction<BytesRef> packedValues) throws IOException {
|
||||
private void writeLeafBlockPackedValues(IndexOutput out, int[] commonPrefixLengths, int count, int sortedDim, IntFunction<BytesRef> packedValues) throws IOException {
|
||||
int prefixLenSum = Arrays.stream(commonPrefixLengths).sum();
|
||||
if (prefixLenSum == packedBytesLength) {
|
||||
// all values in this block are equal
|
||||
|
@ -1109,7 +1316,7 @@ public class BKDWriter implements Closeable {
|
|||
return end - start;
|
||||
}
|
||||
|
||||
protected void writeCommonPrefixes(IndexOutput out, int[] commonPrefixes, byte[] packedValue) throws IOException {
|
||||
private void writeCommonPrefixes(IndexOutput out, int[] commonPrefixes, byte[] packedValue) throws IOException {
|
||||
for(int dim=0;dim<numDims;dim++) {
|
||||
out.writeVInt(commonPrefixes[dim]);
|
||||
//System.out.println(commonPrefixes[dim] + " of " + bytesPerDim);
|
||||
|
@ -1177,7 +1384,7 @@ public class BKDWriter implements Closeable {
|
|||
// TODO: find a way to also checksum this reader? If we changed to markLeftTree, and scanned the final chunk, it could work?
|
||||
try (PointReader reader = source.writer.getReader(source.start + source.count - rightCount, rightCount)) {
|
||||
boolean result = reader.next();
|
||||
assert result;
|
||||
assert result: "rightCount=" + rightCount + " source.count=" + source.count + " source.writer=" + source.writer;
|
||||
System.arraycopy(reader.packedValue(), splitDim*bytesPerDim, scratch1, 0, bytesPerDim);
|
||||
if (numDims > 1) {
|
||||
assert ordBitSet.get(reader.ord()) == false;
|
||||
|
@ -1244,12 +1451,12 @@ public class BKDWriter implements Closeable {
|
|||
|
||||
/* Recursively reorders the provided reader and writes the bkd-tree on the fly. */
|
||||
private void build(int nodeID, int leafNodeOffset,
|
||||
MutablePointValues reader, int from, int to,
|
||||
IndexOutput out,
|
||||
byte[] minPackedValue, byte[] maxPackedValue,
|
||||
byte[] splitPackedValues,
|
||||
long[] leafBlockFPs,
|
||||
int[] spareDocIds) throws IOException {
|
||||
MutablePointValues reader, int from, int to,
|
||||
IndexOutput out,
|
||||
byte[] minPackedValue, byte[] maxPackedValue,
|
||||
byte[] splitPackedValues,
|
||||
long[] leafBlockFPs,
|
||||
int[] spareDocIds) throws IOException {
|
||||
|
||||
if (nodeID >= leafNodeOffset) {
|
||||
// leaf node
|
||||
|
@ -1311,6 +1518,7 @@ public class BKDWriter implements Closeable {
|
|||
for (int i = from; i < to; ++i) {
|
||||
docIDs[i - from] = reader.getDocID(i);
|
||||
}
|
||||
//System.out.println("writeLeafBlock pos=" + out.getFilePointer());
|
||||
writeLeafBlockDocs(out, docIDs, 0, count);
|
||||
|
||||
// Write the common prefixes:
|
||||
|
@ -1344,6 +1552,7 @@ public class BKDWriter implements Closeable {
|
|||
break;
|
||||
}
|
||||
}
|
||||
|
||||
MutablePointsReaderUtils.partition(maxDoc, splitDim, bytesPerDim, commonPrefixLen,
|
||||
reader, from, to, mid, scratchBytesRef1, scratchBytesRef2);
|
||||
|
||||
|
@ -1381,7 +1590,7 @@ public class BKDWriter implements Closeable {
|
|||
for(PathSlice slice : slices) {
|
||||
assert slice.count == slices[0].count;
|
||||
}
|
||||
|
||||
|
||||
if (numDims == 1 && slices[0].writer instanceof OfflinePointWriter && slices[0].count <= maxPointsSortInHeap) {
|
||||
// Special case for 1D, to cutover to heap once we recurse deeply enough:
|
||||
slices[0] = switchToHeap(slices[0], toCloseHeroically);
|
||||
|
|
|
@ -18,7 +18,10 @@ package org.apache.lucene.util.bkd;
|
|||
|
||||
import java.util.List;
|
||||
|
||||
final class HeapPointReader extends PointReader {
|
||||
/** Utility class to read buffered points from in-heap arrays.
|
||||
*
|
||||
* @lucene.internal */
|
||||
public final class HeapPointReader extends PointReader {
|
||||
private int curRead;
|
||||
final List<byte[]> blocks;
|
||||
final int valuesPerBlock;
|
||||
|
@ -30,7 +33,7 @@ final class HeapPointReader extends PointReader {
|
|||
final byte[] scratch;
|
||||
final boolean singleValuePerDoc;
|
||||
|
||||
HeapPointReader(List<byte[]> blocks, int valuesPerBlock, int packedBytesLength, int[] ords, long[] ordsLong, int[] docIDs, int start, int end, boolean singleValuePerDoc) {
|
||||
public HeapPointReader(List<byte[]> blocks, int valuesPerBlock, int packedBytesLength, int[] ords, long[] ordsLong, int[] docIDs, int start, int end, boolean singleValuePerDoc) {
|
||||
this.blocks = blocks;
|
||||
this.valuesPerBlock = valuesPerBlock;
|
||||
this.singleValuePerDoc = singleValuePerDoc;
|
||||
|
|
|
@ -24,18 +24,21 @@ import java.util.List;
|
|||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
final class HeapPointWriter implements PointWriter {
|
||||
int[] docIDs;
|
||||
long[] ordsLong;
|
||||
int[] ords;
|
||||
/** Utility class to write new points into in-heap arrays.
|
||||
*
|
||||
* @lucene.internal */
|
||||
public final class HeapPointWriter implements PointWriter {
|
||||
public int[] docIDs;
|
||||
public long[] ordsLong;
|
||||
public int[] ords;
|
||||
private int nextWrite;
|
||||
private boolean closed;
|
||||
final int maxSize;
|
||||
final int valuesPerBlock;
|
||||
public final int valuesPerBlock;
|
||||
final int packedBytesLength;
|
||||
final boolean singleValuePerDoc;
|
||||
// NOTE: can't use ByteBlockPool because we need random-write access when sorting in heap
|
||||
final List<byte[]> blocks = new ArrayList<>();
|
||||
public final List<byte[]> blocks = new ArrayList<>();
|
||||
|
||||
public HeapPointWriter(int initSize, int maxSize, int packedBytesLength, boolean longOrds, boolean singleValuePerDoc) {
|
||||
docIDs = new int[initSize];
|
||||
|
@ -77,7 +80,7 @@ final class HeapPointWriter implements PointWriter {
|
|||
nextWrite = other.nextWrite;
|
||||
}
|
||||
|
||||
void readPackedValue(int index, byte[] bytes) {
|
||||
public void readPackedValue(int index, byte[] bytes) {
|
||||
assert bytes.length == packedBytesLength;
|
||||
int block = index / valuesPerBlock;
|
||||
int blockIndex = index % valuesPerBlock;
|
||||
|
@ -85,7 +88,7 @@ final class HeapPointWriter implements PointWriter {
|
|||
}
|
||||
|
||||
/** Returns a reference, in <code>result</code>, to the byte[] slice holding this value */
|
||||
void getPackedValueSlice(int index, BytesRef result) {
|
||||
public void getPackedValueSlice(int index, BytesRef result) {
|
||||
int block = index / valuesPerBlock;
|
||||
int blockIndex = index % valuesPerBlock;
|
||||
result.bytes = blocks.get(block);
|
||||
|
@ -138,7 +141,8 @@ final class HeapPointWriter implements PointWriter {
|
|||
@Override
|
||||
public PointReader getReader(long start, long length) {
|
||||
assert start + length <= docIDs.length: "start=" + start + " length=" + length + " docIDs.length=" + docIDs.length;
|
||||
return new HeapPointReader(blocks, valuesPerBlock, packedBytesLength, ords, ordsLong, docIDs, (int) start, nextWrite, singleValuePerDoc);
|
||||
assert start + length <= nextWrite: "start=" + start + " length=" + length + " nextWrite=" + nextWrite;
|
||||
return new HeapPointReader(blocks, valuesPerBlock, packedBytesLength, ords, ordsLong, docIDs, (int) start, Math.toIntExact(start+length), singleValuePerDoc);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -26,13 +26,16 @@ import org.apache.lucene.util.Selector;
|
|||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
final class MutablePointsReaderUtils {
|
||||
/** Utility APIs for sorting and partitioning buffered points.
|
||||
*
|
||||
* @lucene.internal */
|
||||
public final class MutablePointsReaderUtils {
|
||||
|
||||
MutablePointsReaderUtils() {}
|
||||
|
||||
/** Sort the given {@link MutablePointValues} based on its packed value then doc ID. */
|
||||
static void sort(int maxDoc, int packedBytesLength,
|
||||
MutablePointValues reader, int from, int to) {
|
||||
public static void sort(int maxDoc, int packedBytesLength,
|
||||
MutablePointValues reader, int from, int to) {
|
||||
final int bitsPerDocId = PackedInts.bitsRequired(maxDoc - 1);
|
||||
new MSBRadixSorter(packedBytesLength + (bitsPerDocId + 7) / 8) {
|
||||
|
||||
|
@ -88,9 +91,9 @@ final class MutablePointsReaderUtils {
|
|||
}
|
||||
|
||||
/** Sort points on the given dimension. */
|
||||
static void sortByDim(int sortedDim, int bytesPerDim, int[] commonPrefixLengths,
|
||||
MutablePointValues reader, int from, int to,
|
||||
BytesRef scratch1, BytesRef scratch2) {
|
||||
public static void sortByDim(int sortedDim, int bytesPerDim, int[] commonPrefixLengths,
|
||||
MutablePointValues reader, int from, int to,
|
||||
BytesRef scratch1, BytesRef scratch2) {
|
||||
|
||||
// No need for a fancy radix sort here, this is called on the leaves only so
|
||||
// there are not many values to sort
|
||||
|
@ -127,9 +130,9 @@ final class MutablePointsReaderUtils {
|
|||
/** Partition points around {@code mid}. All values on the left must be less
|
||||
* than or equal to it and all values on the right must be greater than or
|
||||
* equal to it. */
|
||||
static void partition(int maxDoc, int splitDim, int bytesPerDim, int commonPrefixLen,
|
||||
MutablePointValues reader, int from, int to, int mid,
|
||||
BytesRef scratch1, BytesRef scratch2) {
|
||||
public static void partition(int maxDoc, int splitDim, int bytesPerDim, int commonPrefixLen,
|
||||
MutablePointValues reader, int from, int to, int mid,
|
||||
BytesRef scratch1, BytesRef scratch2) {
|
||||
final int offset = splitDim * bytesPerDim + commonPrefixLen;
|
||||
final int cmpBytes = bytesPerDim - commonPrefixLen;
|
||||
final int bitsPerDocId = PackedInts.bitsRequired(maxDoc - 1);
|
||||
|
|
|
@ -27,8 +27,10 @@ import org.apache.lucene.store.IndexInput;
|
|||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.LongBitSet;
|
||||
|
||||
/** Reads points from disk in a fixed-with format, previously written with {@link OfflinePointWriter}. */
|
||||
final class OfflinePointReader extends PointReader {
|
||||
/** Reads points from disk in a fixed-with format, previously written with {@link OfflinePointWriter}.
|
||||
*
|
||||
* @lucene.internal */
|
||||
public final class OfflinePointReader extends PointReader {
|
||||
long countLeft;
|
||||
final IndexInput in;
|
||||
private final byte[] packedValue;
|
||||
|
@ -43,7 +45,7 @@ final class OfflinePointReader extends PointReader {
|
|||
// File name we are reading
|
||||
final String name;
|
||||
|
||||
OfflinePointReader(Directory tempDir, String tempFileName, int packedBytesLength, long start, long length,
|
||||
public OfflinePointReader(Directory tempDir, String tempFileName, int packedBytesLength, long start, long length,
|
||||
boolean longOrds, boolean singleValuePerDoc) throws IOException {
|
||||
this.singleValuePerDoc = singleValuePerDoc;
|
||||
int bytesPerDoc = packedBytesLength + Integer.BYTES;
|
||||
|
|
|
@ -26,12 +26,14 @@ import org.apache.lucene.store.Directory;
|
|||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
|
||||
/** Writes points to disk in a fixed-with format. */
|
||||
final class OfflinePointWriter implements PointWriter {
|
||||
/** Writes points to disk in a fixed-with format.
|
||||
*
|
||||
* @lucene.internal */
|
||||
public final class OfflinePointWriter implements PointWriter {
|
||||
|
||||
final Directory tempDir;
|
||||
final IndexOutput out;
|
||||
final String name;
|
||||
public final IndexOutput out;
|
||||
public final String name;
|
||||
final int packedBytesLength;
|
||||
final boolean singleValuePerDoc;
|
||||
long count;
|
||||
|
|
|
@ -24,20 +24,22 @@ import org.apache.lucene.util.LongBitSet;
|
|||
|
||||
/** One pass iterator through all points previously written with a
|
||||
* {@link PointWriter}, abstracting away whether points a read
|
||||
* from (offline) disk or simple arrays in heap. */
|
||||
abstract class PointReader implements Closeable {
|
||||
* from (offline) disk or simple arrays in heap.
|
||||
*
|
||||
* @lucene.internal */
|
||||
public abstract class PointReader implements Closeable {
|
||||
|
||||
/** Returns false once iteration is done, else true. */
|
||||
abstract boolean next() throws IOException;
|
||||
public abstract boolean next() throws IOException;
|
||||
|
||||
/** Returns the packed byte[] value */
|
||||
abstract byte[] packedValue();
|
||||
public abstract byte[] packedValue();
|
||||
|
||||
/** Point ordinal */
|
||||
abstract long ord();
|
||||
public abstract long ord();
|
||||
|
||||
/** DocID for this point */
|
||||
abstract int docID();
|
||||
public abstract int docID();
|
||||
|
||||
/** Iterates through the next {@code count} ords, marking them in the provided {@code ordBitSet}. */
|
||||
public void markOrds(long count, LongBitSet ordBitSet) throws IOException {
|
||||
|
|
|
@ -23,8 +23,10 @@ import java.util.List;
|
|||
|
||||
/** Appends many points, and then at the end provides a {@link PointReader} to iterate
|
||||
* those points. This abstracts away whether we write to disk, or use simple arrays
|
||||
* in heap. */
|
||||
interface PointWriter extends Closeable {
|
||||
* in heap.
|
||||
*
|
||||
* @lucene.internal */
|
||||
public interface PointWriter extends Closeable {
|
||||
/** Add a new point */
|
||||
void append(byte[] packedValue, long ord, int docID) throws IOException;
|
||||
|
||||
|
|
|
@ -621,6 +621,9 @@ public class TestPointQueries extends LuceneTestCase {
|
|||
int numDims = TestUtil.nextInt(random(), 1, PointValues.MAX_DIMENSIONS);
|
||||
|
||||
int sameValuePct = random().nextInt(100);
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: sameValuePct=" + sameValuePct);
|
||||
}
|
||||
|
||||
byte[][][] docValues = new byte[numValues][][];
|
||||
|
||||
|
|
|
@ -16,6 +16,7 @@
|
|||
*/
|
||||
package org.apache.lucene.util.bkd;
|
||||
|
||||
import org.apache.lucene.index.CheckIndex;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.FSDirectory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
|
@ -64,7 +65,10 @@ public class Test2BBKDPoints extends LuceneTestCase {
|
|||
IndexInput in = dir.openInput("1d.bkd", IOContext.DEFAULT);
|
||||
in.seek(indexFP);
|
||||
BKDReader r = new BKDReader(in);
|
||||
r.verify(numDocs);
|
||||
CheckIndex.VerifyPointsVisitor visitor = new CheckIndex.VerifyPointsVisitor("1d", numDocs, r);
|
||||
r.intersect(visitor);
|
||||
assertEquals(r.size(), visitor.getPointCountSeen());
|
||||
assertEquals(r.getDocCount(), visitor.getDocCountSeen());
|
||||
in.close();
|
||||
dir.close();
|
||||
}
|
||||
|
@ -101,7 +105,10 @@ public class Test2BBKDPoints extends LuceneTestCase {
|
|||
IndexInput in = dir.openInput("2d.bkd", IOContext.DEFAULT);
|
||||
in.seek(indexFP);
|
||||
BKDReader r = new BKDReader(in);
|
||||
r.verify(numDocs);
|
||||
CheckIndex.VerifyPointsVisitor visitor = new CheckIndex.VerifyPointsVisitor("2d", numDocs, r);
|
||||
r.intersect(visitor);
|
||||
assertEquals(r.size(), visitor.getPointCountSeen());
|
||||
assertEquals(r.getDocCount(), visitor.getDocCountSeen());
|
||||
in.close();
|
||||
dir.close();
|
||||
}
|
||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.lucene.index.CorruptIndexException;
|
|||
import org.apache.lucene.index.MergeState;
|
||||
import org.apache.lucene.index.PointValues.IntersectVisitor;
|
||||
import org.apache.lucene.index.PointValues.Relation;
|
||||
import org.apache.lucene.index.PointValues;
|
||||
import org.apache.lucene.store.CorruptingIndexOutput;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.FilterDirectory;
|
||||
|
@ -1010,4 +1011,57 @@ public class TestBKD extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
// Claims 16 bytes per dim, but only use the bottom N 1-3 bytes; this would happen e.g. if a user indexes what are actually just short
|
||||
// values as a LongPoint:
|
||||
public void testWastedLeadingBytes() throws Exception {
|
||||
int numDims = TestUtil.nextInt(random(), 1, PointValues.MAX_DIMENSIONS);
|
||||
int bytesPerDim = PointValues.MAX_NUM_BYTES;
|
||||
int bytesUsed = TestUtil.nextInt(random(), 1, 3);
|
||||
|
||||
Directory dir = newFSDirectory(createTempDir());
|
||||
int numDocs = 100000;
|
||||
BKDWriter w = new BKDWriter(numDocs+1, dir, "tmp", numDims, bytesPerDim, 32, 1f, numDocs, true);
|
||||
byte[] tmp = new byte[bytesUsed];
|
||||
byte[] buffer = new byte[numDims * bytesPerDim];
|
||||
for(int i=0;i<numDocs;i++) {
|
||||
for(int dim=0;dim<numDims;dim++) {
|
||||
random().nextBytes(tmp);
|
||||
System.arraycopy(tmp, 0, buffer, dim*bytesPerDim+(bytesPerDim-bytesUsed), tmp.length);
|
||||
}
|
||||
w.add(buffer, i);
|
||||
}
|
||||
|
||||
IndexOutput out = dir.createOutput("bkd", IOContext.DEFAULT);
|
||||
long fp = w.finish(out);
|
||||
out.close();
|
||||
|
||||
IndexInput in = dir.openInput("bkd", IOContext.DEFAULT);
|
||||
in.seek(fp);
|
||||
BKDReader r = new BKDReader(in);
|
||||
int[] count = new int[1];
|
||||
r.intersect(new IntersectVisitor() {
|
||||
|
||||
@Override
|
||||
public void visit(int docID) {
|
||||
count[0]++;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visit(int docID, byte[] packedValue) {
|
||||
visit(docID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Relation compare(byte[] minPacked, byte[] maxPacked) {
|
||||
if (random().nextInt(7) == 1) {
|
||||
return Relation.CELL_CROSSES_QUERY;
|
||||
} else {
|
||||
return Relation.CELL_INSIDE_QUERY;
|
||||
}
|
||||
}
|
||||
});
|
||||
assertEquals(numDocs, count[0]);
|
||||
in.close();
|
||||
dir.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -228,7 +228,7 @@ public class TestFSTs extends LuceneTestCase {
|
|||
final long value = lastOutput + TestUtil.nextInt(random(), 1, 1000);
|
||||
lastOutput = value;
|
||||
pairs.add(new FSTTester.InputOutput<>(terms[idx],
|
||||
outputs.newPair((long) idx, value)));
|
||||
outputs.newPair((long) idx, value)));
|
||||
}
|
||||
new FSTTester<>(random(), dir, inputMode, pairs, outputs, false).doTest(true);
|
||||
}
|
||||
|
|
|
@ -26,7 +26,10 @@ import org.apache.lucene.geo.Rectangle;
|
|||
import org.apache.lucene.index.PointValues.IntersectVisitor;
|
||||
import org.apache.lucene.index.PointValues.Relation;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.SloppyMath;
|
||||
import org.apache.lucene.util.bkd.BKDReader.IndexTree;
|
||||
import org.apache.lucene.util.bkd.BKDReader.IntersectState;
|
||||
import org.apache.lucene.util.bkd.BKDReader;
|
||||
|
||||
import static org.apache.lucene.geo.GeoEncodingUtils.decodeLatitude;
|
||||
|
@ -41,16 +44,16 @@ class NearestNeighbor {
|
|||
|
||||
static class Cell implements Comparable<Cell> {
|
||||
final int readerIndex;
|
||||
final int nodeID;
|
||||
final byte[] minPacked;
|
||||
final byte[] maxPacked;
|
||||
final IndexTree index;
|
||||
|
||||
/** The closest possible distance of all points in this cell */
|
||||
final double distanceMeters;
|
||||
|
||||
public Cell(int readerIndex, int nodeID, byte[] minPacked, byte[] maxPacked, double distanceMeters) {
|
||||
public Cell(IndexTree index, int readerIndex, byte[] minPacked, byte[] maxPacked, double distanceMeters) {
|
||||
this.index = index;
|
||||
this.readerIndex = readerIndex;
|
||||
this.nodeID = nodeID;
|
||||
this.minPacked = minPacked.clone();
|
||||
this.maxPacked = maxPacked.clone();
|
||||
this.distanceMeters = distanceMeters;
|
||||
|
@ -66,7 +69,7 @@ class NearestNeighbor {
|
|||
double minLon = decodeLongitude(minPacked, Integer.BYTES);
|
||||
double maxLat = decodeLatitude(maxPacked, 0);
|
||||
double maxLon = decodeLongitude(maxPacked, Integer.BYTES);
|
||||
return "Cell(readerIndex=" + readerIndex + " lat=" + minLat + " TO " + maxLat + ", lon=" + minLon + " TO " + maxLon + "; distanceMeters=" + distanceMeters + ")";
|
||||
return "Cell(readerIndex=" + readerIndex + " nodeID=" + index.getNodeID() + " isLeaf=" + index.isLeafNode() + " lat=" + minLat + " TO " + maxLat + ", lon=" + minLon + " TO " + maxLon + "; distanceMeters=" + distanceMeters + ")";
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -219,13 +222,21 @@ class NearestNeighbor {
|
|||
List<BKDReader.IntersectState> states = new ArrayList<>();
|
||||
|
||||
// Add root cell for each reader into the queue:
|
||||
int bytesPerDim = -1;
|
||||
|
||||
for(int i=0;i<readers.size();i++) {
|
||||
BKDReader reader = readers.get(i);
|
||||
if (bytesPerDim == -1) {
|
||||
bytesPerDim = reader.getBytesPerDimension();
|
||||
} else if (bytesPerDim != reader.getBytesPerDimension()) {
|
||||
throw new IllegalStateException("bytesPerDim changed from " + bytesPerDim + " to " + reader.getBytesPerDimension() + " across readers");
|
||||
}
|
||||
byte[] minPackedValue = reader.getMinPackedValue();
|
||||
byte[] maxPackedValue = reader.getMaxPackedValue();
|
||||
states.add(reader.getIntersectState(visitor));
|
||||
IntersectState state = reader.getIntersectState(visitor);
|
||||
states.add(state);
|
||||
|
||||
cellQueue.offer(new Cell(i, 1, reader.getMinPackedValue(), reader.getMaxPackedValue(),
|
||||
cellQueue.offer(new Cell(state.index, i, reader.getMinPackedValue(), reader.getMaxPackedValue(),
|
||||
approxBestDistance(minPackedValue, maxPackedValue, pointLat, pointLon)));
|
||||
}
|
||||
|
||||
|
@ -236,12 +247,12 @@ class NearestNeighbor {
|
|||
// TODO: if we replace approxBestDistance with actualBestDistance, we can put an opto here to break once this "best" cell is fully outside of the hitQueue bottom's radius:
|
||||
BKDReader reader = readers.get(cell.readerIndex);
|
||||
|
||||
if (reader.isLeafNode(cell.nodeID)) {
|
||||
if (cell.index.isLeafNode()) {
|
||||
//System.out.println(" leaf");
|
||||
// Leaf block: visit all points and possibly collect them:
|
||||
visitor.curDocBase = docBases.get(cell.readerIndex);
|
||||
visitor.curLiveDocs = liveDocs.get(cell.readerIndex);
|
||||
reader.visitLeafBlockValues(cell.nodeID, states.get(cell.readerIndex));
|
||||
reader.visitLeafBlockValues(cell.index, states.get(cell.readerIndex));
|
||||
//System.out.println(" now " + hitQueue.size() + " hits");
|
||||
} else {
|
||||
//System.out.println(" non-leaf");
|
||||
|
@ -257,14 +268,23 @@ class NearestNeighbor {
|
|||
continue;
|
||||
}
|
||||
|
||||
BytesRef splitValue = BytesRef.deepCopyOf(cell.index.getSplitDimValue());
|
||||
int splitDim = cell.index.getSplitDim();
|
||||
|
||||
// we must clone the index so that we we can recurse left and right "concurrently":
|
||||
IndexTree newIndex = cell.index.clone();
|
||||
byte[] splitPackedValue = cell.maxPacked.clone();
|
||||
reader.copySplitValue(cell.nodeID, splitPackedValue);
|
||||
cellQueue.offer(new Cell(cell.readerIndex, 2*cell.nodeID, cell.minPacked, splitPackedValue,
|
||||
System.arraycopy(splitValue.bytes, splitValue.offset, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
|
||||
|
||||
cell.index.pushLeft();
|
||||
cellQueue.offer(new Cell(cell.index, cell.readerIndex, cell.minPacked, splitPackedValue,
|
||||
approxBestDistance(cell.minPacked, splitPackedValue, pointLat, pointLon)));
|
||||
|
||||
splitPackedValue = cell.minPacked.clone();
|
||||
reader.copySplitValue(cell.nodeID, splitPackedValue);
|
||||
cellQueue.offer(new Cell(cell.readerIndex, 2*cell.nodeID+1, splitPackedValue, cell.maxPacked,
|
||||
System.arraycopy(splitValue.bytes, splitValue.offset, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
|
||||
|
||||
newIndex.pushRight();
|
||||
cellQueue.offer(new Cell(newIndex, cell.readerIndex, splitPackedValue, cell.maxPacked,
|
||||
approxBestDistance(splitPackedValue, cell.maxPacked, pointLat, pointLon)));
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue