package org.apache.lucene.util.bkd;
import java.io.IOException;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.PointValues;
import org.apache.lucene.search.DocIdSetIterator;
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.FutureArrays;
import org.apache.lucene.util.MathUtil;
public final class BKDReader extends PointValues implements Accountable {
final int leafNodeOffset;
final int numDataDims;
final int numIndexDims;
final int bytesPerDim;
final int numLeaves;
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;
protected final int packedIndexBytesLength;
final byte[] packedIndex;
public BKDReader(IndexInput in) throws IOException {
version = CodecUtil.checkHeader(in, BKDWriter.CODEC_NAME, BKDWriter.VERSION_START, BKDWriter.VERSION_CURRENT);
numDataDims = in.readVInt();
if (version >= BKDWriter.VERSION_SELECTIVE_INDEXING) {
numIndexDims = in.readVInt();
} else {
numIndexDims = numDataDims;
}
maxPointsInLeafNode = in.readVInt();
bytesPerDim = in.readVInt();
packedBytesLength = numDataDims * bytesPerDim;
packedIndexBytesLength = numIndexDims * bytesPerDim;
numLeaves = in.readVInt();
assert numLeaves > 0;
leafNodeOffset = numLeaves;
minPackedValue = new byte[packedIndexBytesLength];
maxPackedValue = new byte[packedIndexBytesLength];
in.readBytes(minPackedValue, 0, packedIndexBytesLength);
in.readBytes(maxPackedValue, 0, packedIndexBytesLength);
for(int dim=0;dim<numIndexDims;dim++) {
if (FutureArrays.compareUnsigned(minPackedValue, dim * bytesPerDim, dim * bytesPerDim + bytesPerDim, maxPackedValue, dim * bytesPerDim, dim * bytesPerDim + bytesPerDim) > 0) {
throw new CorruptIndexException("minPackedValue " + new BytesRef(minPackedValue) + " is > maxPackedValue " + new BytesRef(maxPackedValue) + " for dim=" + dim, in);
}
}
pointCount = in.readVLong();
docCount = in.readVInt();
int numBytes = in.readVInt();
packedIndex = new byte[numBytes];
in.readBytes(packedIndex, 0, numBytes);
this.in = in;
}
long getMinLeafBlockFP() {
return new ByteArrayDataInput(packedIndex).readVLong();
}
public class IndexTree implements Cloneable {
private int nodeID;
private int level;
private int splitDim;
private final byte[][] splitPackedValueStack;
private final ByteArrayDataInput in;
private final long[] leafBlockFPStack;
private final int[] leftNodePositions;
private final int[] rightNodePositions;
private final int[] splitDims;
private final boolean[] negativeDeltas;
private final byte[][] splitValuesStack;
private final BytesRef scratch;
IndexTree() {
int treeDepth = getTreeDepth();
splitPackedValueStack = new byte[treeDepth+1][];
nodeID = 1;
level = 1;
splitPackedValueStack[level] = new byte[packedIndexBytesLength];
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[numIndexDims*(treeDepth+1)];
in = new ByteArrayDataInput(packedIndex);
splitValuesStack[0] = new byte[packedIndexBytesLength];
readNodeData(false);
scratch = new BytesRef();
scratch.length = bytesPerDim;
}
public void pushLeft() {
int nodePosition = leftNodePositions[level];
nodeID *= 2;
level++;
if (splitPackedValueStack[level] == null) {
splitPackedValueStack[level] = new byte[packedIndexBytesLength];
}
System.arraycopy(negativeDeltas, (level-1)*numIndexDims, negativeDeltas, level*numIndexDims, numIndexDims);
assert splitDim != -1;
negativeDeltas[level*numIndexDims+splitDim] = true;
in.setPosition(nodePosition);
readNodeData(true);
}
@Override
public IndexTree clone() {
IndexTree index = new IndexTree();
index.nodeID = nodeID;
index.level = level;
index.splitDim = splitDim;
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*numIndexDims, index.negativeDeltas, level*numIndexDims, numIndexDims);
index.splitDims[level] = splitDims[level];
return index;
}
public void pushRight() {
int nodePosition = rightNodePositions[level];
nodeID = nodeID * 2 + 1;
level++;
if (splitPackedValueStack[level] == null) {
splitPackedValueStack[level] = new byte[packedIndexBytesLength];
}
System.arraycopy(negativeDeltas, (level-1)*numIndexDims, negativeDeltas, level*numIndexDims, numIndexDims);
assert splitDim != -1;
negativeDeltas[level*numIndexDims+splitDim] = false;
in.setPosition(nodePosition);
readNodeData(false);
}
public void pop() {
nodeID /= 2;
level--;
splitDim = splitDims[level];
}
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];
}
public int getSplitDim() {
assert isLeafNode() == false;
return splitDim;
}
public BytesRef getSplitDimValue() {
assert isLeafNode() == false;
scratch.bytes = splitValuesStack[level];
scratch.offset = splitDim * bytesPerDim;
return scratch;
}
public long getLeafBlockFP() {
assert isLeafNode(): "nodeID=" + nodeID + " is not a leaf";
return leafBlockFPStack[level];
}
public int getNumLeaves() {
int leftMostLeafNode = nodeID;
while (leftMostLeafNode < leafNodeOffset) {
leftMostLeafNode = leftMostLeafNode * 2;
}
int rightMostLeafNode = nodeID;
while (rightMostLeafNode < leafNodeOffset) {
rightMostLeafNode = rightMostLeafNode * 2 + 1;
}
final int numLeaves;
if (rightMostLeafNode >= leftMostLeafNode) {
numLeaves = rightMostLeafNode - leftMostLeafNode + 1;
} else {
numLeaves = rightMostLeafNode - leftMostLeafNode + 1 + leafNodeOffset;
}
assert numLeaves == getNumLeavesSlow(nodeID) : numLeaves + " " + getNumLeavesSlow(nodeID);
return numLeaves;
}
private int getNumLeavesSlow(int node) {
if (node >= 2 * leafNodeOffset) {
return 0;
} else if (node >= leafNodeOffset) {
return 1;
} else {
final int leftCount = getNumLeavesSlow(node * 2);
final int rightCount = getNumLeavesSlow(node * 2 + 1);
return leftCount + rightCount;
}
}
private void readNodeData(boolean isLeft) {
leafBlockFPStack[level] = leafBlockFPStack[level-1];
if (isLeft == false) {
leafBlockFPStack[level] += in.readVLong();
}
if (isLeafNode()) {
splitDim = -1;
} else {
int code = in.readVInt();
splitDim = code % numIndexDims;
splitDims[level] = splitDim;
code /= numIndexDims;
int prefix = code % (1+bytesPerDim);
int suffix = bytesPerDim - prefix;
if (splitValuesStack[level] == null) {
splitValuesStack[level] = new byte[packedIndexBytesLength];
}
System.arraycopy(splitValuesStack[level-1], 0, splitValuesStack[level], 0, packedIndexBytesLength);
if (suffix > 0) {
int firstDiffByteDelta = code / (1+bytesPerDim);
if (negativeDeltas[level*numIndexDims + 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 {
}
int leftNumBytes;
if (nodeID * 2 < leafNodeOffset) {
leftNumBytes = in.readVInt();
} else {
leftNumBytes = 0;
}
leftNodePositions[level] = in.getPosition();
rightNodePositions[level] = leftNodePositions[level] + leftNumBytes;
}
}
}
private int getTreeDepth() {
return MathUtil.log(numLeaves, 2) + 2;
}
public static final class IntersectState {
final IndexInput in;
final BKDReaderDocIDSetIterator scratchIterator;
final byte[] scratchDataPackedValue, scratchMinIndexPackedValue, scratchMaxIndexPackedValue;
final int[] commonPrefixLengths;
final IntersectVisitor visitor;
public final IndexTree index;
public IntersectState(IndexInput in, int numDims,
int packedBytesLength,
int packedIndexBytesLength,
int maxPointsInLeafNode,
IntersectVisitor visitor,
IndexTree indexVisitor) {
this.in = in;
this.visitor = visitor;
this.commonPrefixLengths = new int[numDims];
this.scratchIterator = new BKDReaderDocIDSetIterator(maxPointsInLeafNode);
this.scratchDataPackedValue = new byte[packedBytesLength];
this.scratchMinIndexPackedValue = new byte[packedIndexBytesLength];
this.scratchMaxIndexPackedValue = new byte[packedIndexBytesLength];
this.index = indexVisitor;
}
}
@Override
public void intersect(IntersectVisitor visitor) throws IOException {
intersect(getIntersectState(visitor), minPackedValue, maxPackedValue);
}
@Override
public long estimatePointCount(IntersectVisitor visitor) {
return estimatePointCount(getIntersectState(visitor), minPackedValue, maxPackedValue);
}
private void addAll(IntersectState state, boolean grown) throws IOException {
if (grown == false) {
final long maxPointCount = (long) maxPointsInLeafNode * state.index.getNumLeaves();
if (maxPointCount <= Integer.MAX_VALUE) {
state.visitor.grow((int) maxPointCount);
grown = true;
}
}
if (state.index.isLeafNode()) {
assert grown;
if (state.index.nodeExists()) {
visitDocIDs(state.in, state.index.getLeafBlockFP(), state.visitor);
}
} else {
state.index.pushLeft();
addAll(state, grown);
state.index.pop();
state.index.pushRight();
addAll(state, grown);
state.index.pop();
}
}
public IntersectState getIntersectState(IntersectVisitor visitor) {
IndexTree index = new IndexTree();
return new IntersectState(in.clone(), numDataDims,
packedBytesLength,
packedIndexBytesLength,
maxPointsInLeafNode,
visitor,
index);
}
public void visitLeafBlockValues(IndexTree index, IntersectState state) throws IOException {
int count = readDocIDs(state.in, index.getLeafBlockFP(), state.scratchIterator);
visitDocValues(state.commonPrefixLengths, state.scratchDataPackedValue, state.scratchMinIndexPackedValue, state.scratchMaxIndexPackedValue, state.in, state.scratchIterator, count, state.visitor);
}
private void visitDocIDs(IndexInput in, long blockFP, IntersectVisitor visitor) throws IOException {
in.seek(blockFP);
int count = in.readVInt();
DocIdsWriter.readInts(in, count, visitor);
}
int readDocIDs(IndexInput in, long blockFP, BKDReaderDocIDSetIterator iterator) throws IOException {
in.seek(blockFP);
int count = in.readVInt();
DocIdsWriter.readInts(in, count, iterator.docIDs);
return count;
}
void visitDocValues(int[] commonPrefixLengths, byte[] scratchDataPackedValue, byte[] scratchMinIndexPackedValue, byte[] scratchMaxIndexPackedValue,
IndexInput in, BKDReaderDocIDSetIterator scratchIterator, int count, IntersectVisitor visitor) throws IOException {
if (version >= BKDWriter.VERSION_LOW_CARDINALITY_LEAVES) {
visitDocValuesWithCardinality(commonPrefixLengths, scratchDataPackedValue, scratchMinIndexPackedValue, scratchMaxIndexPackedValue, in, scratchIterator, count, visitor);
} else {
visitDocValuesNoCardinality(commonPrefixLengths, scratchDataPackedValue, scratchMinIndexPackedValue, scratchMaxIndexPackedValue, in, scratchIterator, count, visitor);
}
}
void visitDocValuesNoCardinality(int[] commonPrefixLengths, byte[] scratchDataPackedValue, byte[] scratchMinIndexPackedValue, byte[] scratchMaxIndexPackedValue,
IndexInput in, BKDReaderDocIDSetIterator scratchIterator, int count, IntersectVisitor visitor) throws IOException {
readCommonPrefixes(commonPrefixLengths, scratchDataPackedValue, in);
if (numIndexDims != 1 && version >= BKDWriter.VERSION_LEAF_STORES_BOUNDS) {
byte[] minPackedValue = scratchMinIndexPackedValue;
System.arraycopy(scratchDataPackedValue, 0, minPackedValue, 0, packedIndexBytesLength);
byte[] maxPackedValue = scratchMaxIndexPackedValue;
System.arraycopy(minPackedValue, 0, maxPackedValue, 0, packedIndexBytesLength);
readMinMax(commonPrefixLengths, minPackedValue, maxPackedValue, in);
Relation r = visitor.compare(minPackedValue, maxPackedValue);
if (r == Relation.CELL_OUTSIDE_QUERY) {
return;
}
visitor.grow(count);
if (r == Relation.CELL_INSIDE_QUERY) {
for (int i = 0; i < count; ++i) {
visitor.visit(scratchIterator.docIDs[i]);
}
return;
}
} else {
visitor.grow(count);
}
int compressedDim = readCompressedDim(in);
if (compressedDim == -1) {
visitUniqueRawDocValues(scratchDataPackedValue, scratchIterator, count, visitor);
} else {
visitCompressedDocValues(commonPrefixLengths, scratchDataPackedValue, in, scratchIterator, count, visitor, compressedDim);
}
}
void visitDocValuesWithCardinality(int[] commonPrefixLengths, byte[] scratchDataPackedValue, byte[] scratchMinIndexPackedValue, byte[] scratchMaxIndexPackedValue,
IndexInput in, BKDReaderDocIDSetIterator scratchIterator, int count, IntersectVisitor visitor) throws IOException {
readCommonPrefixes(commonPrefixLengths, scratchDataPackedValue, in);
int compressedDim = readCompressedDim(in);
if (compressedDim == -1) {
visitor.grow(count);
visitUniqueRawDocValues(scratchDataPackedValue, scratchIterator, count, visitor);
} else {
if (numIndexDims != 1) {
byte[] minPackedValue = scratchMinIndexPackedValue;
System.arraycopy(scratchDataPackedValue, 0, minPackedValue, 0, packedIndexBytesLength);
byte[] maxPackedValue = scratchMaxIndexPackedValue;
System.arraycopy(minPackedValue, 0, maxPackedValue, 0, packedIndexBytesLength);
readMinMax(commonPrefixLengths, minPackedValue, maxPackedValue, in);
Relation r = visitor.compare(minPackedValue, maxPackedValue);
if (r == Relation.CELL_OUTSIDE_QUERY) {
return;
}
visitor.grow(count);
if (r == Relation.CELL_INSIDE_QUERY) {
for (int i = 0; i < count; ++i) {
visitor.visit(scratchIterator.docIDs[i]);
}
return;
}
} else {
visitor.grow(count);
}
if (compressedDim == -2) {
visitSparseRawDocValues(commonPrefixLengths, scratchDataPackedValue, in, scratchIterator, count, visitor);
} else {
visitCompressedDocValues(commonPrefixLengths, scratchDataPackedValue, in, scratchIterator, count, visitor, compressedDim);
}
}
}
private void readMinMax(int[] commonPrefixLengths, byte[] minPackedValue, byte[] maxPackedValue, IndexInput in) throws IOException {
for (int dim = 0; dim < numIndexDims; dim++) {
int prefix = commonPrefixLengths[dim];
in.readBytes(minPackedValue, dim * bytesPerDim + prefix, bytesPerDim - prefix);
in.readBytes(maxPackedValue, dim * bytesPerDim + prefix, bytesPerDim - prefix);
}
}
private void visitSparseRawDocValues(int[] commonPrefixLengths, byte[] scratchPackedValue, IndexInput in, BKDReaderDocIDSetIterator scratchIterator, int count, IntersectVisitor visitor) throws IOException {
int i;
for (i = 0; i < count;) {
int length = in.readVInt();
for(int dim = 0; dim < numDataDims; dim++) {
int prefix = commonPrefixLengths[dim];
in.readBytes(scratchPackedValue, dim*bytesPerDim + prefix, bytesPerDim - prefix);
}
scratchIterator.reset(i, length);
visitor.visit(scratchIterator, scratchPackedValue);
i += length;
}
if (i != count) {
throw new CorruptIndexException("Sub blocks do not add up to the expected count: " + count + " != " + i, in);
}
}
private void visitUniqueRawDocValues(byte[] scratchPackedValue, BKDReaderDocIDSetIterator scratchIterator, int count, IntersectVisitor visitor) throws IOException {
scratchIterator.reset(0, count);
visitor.visit(scratchIterator, scratchPackedValue);
}
private void visitCompressedDocValues(int[] commonPrefixLengths, byte[] scratchPackedValue, IndexInput in, BKDReaderDocIDSetIterator scratchIterator, int count, IntersectVisitor visitor, int compressedDim) throws IOException {
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 < numDataDims; dim++) {
int prefix = commonPrefixLengths[dim];
in.readBytes(scratchPackedValue, dim*bytesPerDim + prefix, bytesPerDim - prefix);
}
visitor.visit(scratchIterator.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 < -2 || compressedDim >= numDataDims || (version < BKDWriter.VERSION_LOW_CARDINALITY_LEAVES && compressedDim == -2)) {
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<numDataDims;dim++) {
int prefix = in.readVInt();
commonPrefixLengths[dim] = prefix;
if (prefix > 0) {
in.readBytes(scratchPackedValue, dim*bytesPerDim, prefix);
}
}
}
private void intersect(IntersectState state, byte[] cellMinPacked, byte[] cellMaxPacked) throws IOException {
Relation r = state.visitor.compare(cellMinPacked, cellMaxPacked);
if (r == Relation.CELL_OUTSIDE_QUERY) {
} else if (r == Relation.CELL_INSIDE_QUERY) {
addAll(state, false);
} else if (state.index.isLeafNode()) {
if (state.index.nodeExists()) {
int count = readDocIDs(state.in, state.index.getLeafBlockFP(), state.scratchIterator);
visitDocValues(state.commonPrefixLengths, state.scratchDataPackedValue, state.scratchMinIndexPackedValue, state.scratchMaxIndexPackedValue, state.in, state.scratchIterator, count, state.visitor);
}
} else {
int splitDim = state.index.getSplitDim();
assert splitDim >= 0: "splitDim=" + splitDim + ", numIndexDims=" + numIndexDims;
assert splitDim < numIndexDims: "splitDim=" + splitDim + ", numIndexDims=" + numIndexDims;
byte[] splitPackedValue = state.index.getSplitPackedValue();
BytesRef splitDimValue = state.index.getSplitDimValue();
assert splitDimValue.length == bytesPerDim;
assert FutureArrays.compareUnsigned(cellMinPacked, splitDim * bytesPerDim, splitDim * bytesPerDim + bytesPerDim, splitDimValue.bytes, splitDimValue.offset, splitDimValue.offset + bytesPerDim) <= 0: "bytesPerDim=" + bytesPerDim + " splitDim=" + splitDim + " numIndexDims=" + numIndexDims + " numDataDims=" + numDataDims;
assert FutureArrays.compareUnsigned(cellMaxPacked, splitDim * bytesPerDim, splitDim * bytesPerDim + bytesPerDim, splitDimValue.bytes, splitDimValue.offset, splitDimValue.offset + bytesPerDim) >= 0: "bytesPerDim=" + bytesPerDim + " splitDim=" + splitDim + " numIndexDims=" + numIndexDims + " numDataDims=" + numDataDims;
System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, packedIndexBytesLength);
System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
state.index.pushLeft();
intersect(state, cellMinPacked, splitPackedValue);
state.index.pop();
System.arraycopy(splitPackedValue, splitDim*bytesPerDim, splitDimValue.bytes, splitDimValue.offset, bytesPerDim);
System.arraycopy(cellMinPacked, 0, splitPackedValue, 0, packedIndexBytesLength);
System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
state.index.pushRight();
intersect(state, splitPackedValue, cellMaxPacked);
state.index.pop();
}
}
private long estimatePointCount(IntersectState state, byte[] cellMinPacked, byte[] cellMaxPacked) {
Relation r = state.visitor.compare(cellMinPacked, cellMaxPacked);
if (r == Relation.CELL_OUTSIDE_QUERY) {
return 0L;
} else if (r == Relation.CELL_INSIDE_QUERY) {
return (long) maxPointsInLeafNode * state.index.getNumLeaves();
} else if (state.index.isLeafNode()) {
return (maxPointsInLeafNode + 1) / 2;
} else {
int splitDim = state.index.getSplitDim();
assert splitDim >= 0: "splitDim=" + splitDim + ", numIndexDims=" + numIndexDims;
assert splitDim < numIndexDims: "splitDim=" + splitDim + ", numIndexDims=" + numIndexDims;
byte[] splitPackedValue = state.index.getSplitPackedValue();
BytesRef splitDimValue = state.index.getSplitDimValue();
assert splitDimValue.length == bytesPerDim;
assert FutureArrays.compareUnsigned(cellMinPacked, splitDim * bytesPerDim, splitDim * bytesPerDim + bytesPerDim, splitDimValue.bytes, splitDimValue.offset, splitDimValue.offset + bytesPerDim) <= 0: "bytesPerDim=" + bytesPerDim + " splitDim=" + splitDim + " numIndexDims=" + numIndexDims + " numDataDims=" + numDataDims;
assert FutureArrays.compareUnsigned(cellMaxPacked, splitDim * bytesPerDim, splitDim * bytesPerDim + bytesPerDim, splitDimValue.bytes, splitDimValue.offset, splitDimValue.offset + bytesPerDim) >= 0: "bytesPerDim=" + bytesPerDim + " splitDim=" + splitDim + " numIndexDims=" + numIndexDims + " numDataDims=" + numDataDims;
System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, packedIndexBytesLength);
System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
state.index.pushLeft();
final long leftCost = estimatePointCount(state, cellMinPacked, splitPackedValue);
state.index.pop();
System.arraycopy(splitPackedValue, splitDim*bytesPerDim, splitDimValue.bytes, splitDimValue.offset, bytesPerDim);
System.arraycopy(cellMinPacked, 0, splitPackedValue, 0, packedIndexBytesLength);
System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
state.index.pushRight();
final long rightCost = estimatePointCount(state, splitPackedValue, cellMaxPacked);
state.index.pop();
return leftCost + rightCost;
}
}
@Override
public long ramBytesUsed() {
return packedIndex.length;
}
@Override
public byte[] getMinPackedValue() {
return minPackedValue.clone();
}
@Override
public byte[] getMaxPackedValue() {
return maxPackedValue.clone();
}
@Override
public int getNumDataDimensions() {
return numDataDims;
}
@Override
public int getNumIndexDimensions() {
return numIndexDims;
}
@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;
}
protected static class BKDReaderDocIDSetIterator extends DocIdSetIterator {
private int idx;
private int length;
private int offset;
private int docID;
final int[] docIDs;
public BKDReaderDocIDSetIterator(int maxPointsInLeafNode) {
this.docIDs = new int[maxPointsInLeafNode];
}
@Override
public int docID() {
return docID;
}
private void reset(int offset, int length) {
this.offset = offset;
this.length = length;
assert offset + length <= docIDs.length;
this.docID = -1;
this.idx = 0;
}
@Override
public int nextDoc() throws IOException {
if (idx == length) {
docID = DocIdSetIterator.NO_MORE_DOCS;
} else {
docID = docIDs[offset + idx];
idx++;
}
return docID;
}
@Override
public int advance(int target) throws IOException {
return slowAdvance(target);
}
@Override
public long cost() {
return length;
}
}
}