You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2015/10/07 10:34:30 UTC
svn commit: r1707203 - in /lucene/dev/branches/lucene6825/lucene/core/src:
java/org/apache/lucene/util/ java/org/apache/lucene/util/bkd/
test/org/apache/lucene/util/bkd/
Author: mikemccand
Date: Wed Oct 7 08:34:30 2015
New Revision: 1707203
URL: http://svn.apache.org/viewvc?rev=1707203&view=rev
Log:
LUCENE-6825: starting patch
Added:
lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/
lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/BKDReader.java (with props)
lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java (with props)
lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/HeapReader.java (with props)
lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/HeapWriter.java (with props)
lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/OfflineReader.java (with props)
lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/OfflineWriter.java (with props)
lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/Reader.java (with props)
lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/Util.java (with props)
lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/Writer.java (with props)
lucene/dev/branches/lucene6825/lucene/core/src/test/org/apache/lucene/util/bkd/
lucene/dev/branches/lucene6825/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKD.java (with props)
Modified:
lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
Modified: lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java?rev=1707203&r1=1707202&r2=1707203&view=diff
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java (original)
+++ lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java Wed Oct 7 08:34:30 2015
@@ -49,6 +49,8 @@ import java.util.Locale;
*/
public final class OfflineSorter {
+ // nocommit cutover to Directory API
+
private static Path DEFAULT_TEMP_DIR;
/** Convenience constant for megabytes */
Added: lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/BKDReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/BKDReader.java?rev=1707203&view=auto
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/BKDReader.java (added)
+++ lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/BKDReader.java Wed Oct 7 08:34:30 2015
@@ -0,0 +1,263 @@
+package org.apache.lucene.util.bkd;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.DocIdSetBuilder;
+import org.apache.lucene.util.RamUsageEstimator;
+
+// nocommit rename generic stuff (Util, Reader, Writer) w/ BKD prefix even though they are package private
+
+// nocommit the try/finally/delete-stuff is frustrating
+
+/** Handles intersection of an multi-dimensional shape in byte[] space with a block KD-tree previously written with {@link BKDWriter}.
+ *
+ * @lucene.experimental */
+
+public final class BKDReader implements Accountable {
+ // Packed array of byte[] holding all split values in the full binary tree:
+ final private byte[] splitPackedValues;
+ final private long[] leafBlockFPs;
+ final private int leafNodeOffset;
+ final int maxDoc;
+ final int numDims;
+ final int bytesPerDim;
+ final IndexInput in;
+ final int packedBytesLength;
+ final int maxPointsInLeafNode;
+
+ enum Relation {CELL_INSIDE_QUERY, QUERY_CROSSES_CELL, QUERY_OUTSIDE_CELL};
+
+ /** We recurse the BKD tree, using a provided instance of this to guide the recursion.
+ *
+ * @lucene.experimental */
+ interface IntersectVisitor {
+ /** Called for all docs in a leaf cell that's fully contained by the query */
+ void visit(int docID);
+
+ /** Called for all docs in a leaf cell that crosses the query */
+ void visit(int docID, byte[] packedValue);
+
+ /** Called for non-leaf cells to test how the cell relates to the query */
+ Relation compare(byte[] minPacked, byte[] maxPacked);
+ }
+
+ /** Caller must pre-seek the provided {@link IndexInput} to the index location that {@link BKDWriter#finish} returned */
+ public BKDReader(IndexInput in, int maxDoc) throws IOException {
+
+ numDims = in.readVInt();
+ maxPointsInLeafNode = in.readVInt();
+ bytesPerDim = in.readVInt();
+ packedBytesLength = numDims * bytesPerDim;
+ System.out.println("R: packed=" + bytesPerDim);
+
+ // Read index:
+ int numLeaves = in.readVInt();
+ leafNodeOffset = numLeaves;
+
+ splitPackedValues = new byte[(1+bytesPerDim)*numLeaves];
+ in.readBytes(splitPackedValues, 0, splitPackedValues.length);
+
+ // Tree is fully balanced binary tree, so number of nodes = numLeaves-1, except our nodeIDs are 1-based (splitPackedValues[0] is unused):
+ leafBlockFPs = new long[numLeaves];
+ for(int i=0;i<numLeaves;i++) {
+ leafBlockFPs[i] = in.readVLong();
+ }
+
+ this.maxDoc = maxDoc;
+ this.in = in;
+ }
+
+ private static final class IntersectState {
+ final IndexInput in;
+ final int[] scratchDocIDs;
+ final byte[] scratchPackedValue;
+
+ // Minimum point of the N-dim rect containing the query shape:
+ final byte[] minPacked;
+ // Maximum point of the N-dim rect containing the query shape:
+ final byte[] maxPacked;
+ final IntersectVisitor visitor;
+
+ public IntersectState(IndexInput in, int packedBytesLength, int maxDoc,
+ int maxPointsInLeafNode, byte[] minPacked, byte[] maxPacked,
+ IntersectVisitor visitor) {
+ this.in = in;
+ this.minPacked = minPacked;
+ this.maxPacked = maxPacked;
+ this.visitor = visitor;
+ this.scratchDocIDs = new int[maxPointsInLeafNode];
+ this.scratchPackedValue = new byte[packedBytesLength];
+ }
+ }
+
+ public void intersect(IntersectVisitor visitor) throws IOException {
+ byte[] minPacked = new byte[packedBytesLength];
+ byte[] maxPacked = new byte[packedBytesLength];
+ Arrays.fill(maxPacked, (byte) 0xff);
+ intersect(minPacked, maxPacked, visitor);
+ }
+
+ public void intersect(byte[] minPacked, byte[] maxPacked, IntersectVisitor visitor) throws IOException {
+ IntersectState state = new IntersectState(in.clone(), packedBytesLength, maxDoc,
+ maxPointsInLeafNode, minPacked, maxPacked,
+ visitor);
+ byte[] rootMinPacked = new byte[packedBytesLength];
+ byte[] rootMaxPacked = new byte[packedBytesLength];
+ Arrays.fill(rootMaxPacked, (byte) 0xff);
+ intersect(state, 1, rootMinPacked, rootMaxPacked);
+ }
+
+ /** 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("R: leaf");
+
+ // Leaf node
+ state.in.seek(leafBlockFPs[nodeID-leafNodeOffset]);
+
+ // How many points are stored in this leaf cell:
+ int count = state.in.readVInt();
+ // nocommit can we maybe get this back, if it's a perf win? Note that we can also grow "up above" when addAll is first entered...
+ //state.docs.grow(count);
+ int docID = 0;
+ for(int i=0;i<count;i++) {
+ docID += state.in.readVInt();
+ state.visitor.visit(docID);
+ }
+ } else {
+ addAll(state, 2*nodeID);
+ addAll(state, 2*nodeID+1);
+ }
+ }
+
+ private void intersect(IntersectState state,
+ int nodeID,
+ byte[] cellMinPacked, byte[] cellMaxPacked)
+ throws IOException {
+
+ //System.out.println("\nR: intersect nodeID=" + nodeID + " cellMin=" + bytesToInt(cellMinPacked, 0) + " cellMax=" + bytesToInt(cellMaxPacked, 0));
+
+ // Optimization: only check the visitor when the current cell does not fully contain the bbox. E.g. if the
+ // query is a small area around London, UK, most of the high nodes in the BKD tree as we recurse will fully
+ // contain the query, so we quickly recurse down until the nodes cross the query:
+ boolean cellContainsQuery = Util.contains(bytesPerDim,
+ cellMinPacked, cellMaxPacked,
+ state.minPacked, state.maxPacked);
+
+ //System.out.println("R: cellContainsQuery=" + cellContainsQuery);
+
+ if (cellContainsQuery == false) {
+
+ Relation r = state.visitor.compare(cellMinPacked, cellMaxPacked);
+ //System.out.println("R: relation=" + r);
+
+ if (r == Relation.QUERY_OUTSIDE_CELL) {
+ // 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, so we fall through and do full filtering
+ }
+ }
+
+ if (nodeID >= leafNodeOffset) {
+ // Leaf node; scan and filter all points in this block:
+ //System.out.println(" intersect leaf nodeID=" + nodeID + " vs leafNodeOffset=" + leafNodeOffset + " fp=" + leafBlockFPs[nodeID-leafNodeOffset]);
+
+ state.in.seek(leafBlockFPs[nodeID-leafNodeOffset]);
+
+ // How many points are stored in this leaf cell:
+ int count = state.in.readVInt();
+
+ // nocommit can we get this back?
+ //state.docs.grow(count);
+ int docID = 0;
+ for(int i=0;i<count;i++) {
+ docID += state.in.readVInt();
+ state.scratchDocIDs[i] = docID;
+ }
+
+ // Again, this time reading values and checking with the visitor
+ for(int i=0;i<count;i++) {
+ state.in.readBytes(state.scratchPackedValue, 0, state.scratchPackedValue.length);
+ state.visitor.visit(state.scratchDocIDs[i], state.scratchPackedValue);
+ }
+
+ } else {
+
+ // Non-leaf node: recurse on the split left and right nodes
+
+ int address = nodeID * (bytesPerDim+1);
+ int splitDim = splitPackedValues[address] & 0xff;
+ assert splitDim < numDims;
+
+ // nocommit can we alloc & reuse this up front?
+ byte[] splitValue = new byte[bytesPerDim];
+ System.arraycopy(splitPackedValues, address+1, splitValue, 0, bytesPerDim);
+
+ byte[] splitPackedValue = new byte[packedBytesLength];
+
+ if (Util.compare(bytesPerDim, state.minPacked, splitDim, splitValue, 0) <= 0) {
+ // The query bbox overlaps our left cell, so we must recurse:
+ System.arraycopy(state.maxPacked, 0, splitPackedValue, 0, packedBytesLength);
+ System.arraycopy(splitValue, 0, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
+ intersect(state,
+ 2*nodeID,
+ cellMinPacked, splitPackedValue);
+ }
+
+ if (Util.compare(bytesPerDim, state.maxPacked, splitDim, splitValue, 0) >= 0) {
+ // The query bbox overlaps our left cell, so we must recurse:
+ System.arraycopy(state.minPacked, 0, splitPackedValue, 0, packedBytesLength);
+ System.arraycopy(splitValue, 0, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
+ intersect(state,
+ 2*nodeID+1,
+ splitPackedValue, cellMaxPacked);
+ }
+ }
+ }
+
+ // nocommit removeme
+ static int bytesToInt(byte[] src, int index) {
+ int x = 0;
+ for(int i=0;i<4;i++) {
+ x |= (src[4*index+i] & 0xff) << (24-i*8);
+ }
+ // Re-flip the sign bit to restore the original value:
+ return x ^ 0x80000000;
+ }
+
+ @Override
+ public long ramBytesUsed() {
+ return splitPackedValues.length +
+ leafBlockFPs.length * RamUsageEstimator.NUM_BYTES_LONG;
+ }
+}
Added: lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java?rev=1707203&view=auto
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java (added)
+++ lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java Wed Oct 7 08:34:30 2015
@@ -0,0 +1,768 @@
+package org.apache.lucene.util.bkd;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.nio.file.DirectoryStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Comparator;
+
+import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.ByteArrayDataOutput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.RAMOutputStream;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.InPlaceMergeSorter;
+import org.apache.lucene.util.LongBitSet;
+import org.apache.lucene.util.OfflineSorter.ByteSequencesWriter;
+import org.apache.lucene.util.OfflineSorter;
+import org.apache.lucene.util.RamUsageEstimator;
+
+// TODO
+// - the compression is somewhat stupid now (delta vInt for 1024 docIDs, no compression for the byte[] values even though they have high locality)
+// - allow variable length byte[], but this is quite a bit more hairy
+// - we could also index "auto-prefix terms" here, and use better compression, and maybe only use for the "fully contained" case so we'd
+// only index docIDs
+// - the index could be efficiently encoded as an FST, so we don't have wasteful
+// (monotonic) long[] leafBlockFPs; or we could use MonotonicLongValues ... but then
+// the index is already plenty small: 60M OSM points --> 1.1 MB with 128 points
+// per leaf, and you can reduce that by putting more points per leaf
+// - we could use threads while building; the higher nodes are very parallelizable
+
+/** Recursively builds a block KD-tree to assign all incoming points in N-dim space to smaller
+ * and smaller N-dim rectangles (cells) until the number of points in a given
+ * rectangle is <= <code>maxPointsInLeafNode</code>. The tree is
+ * fully balanced, which means the leaf nodes will have between 50% and 100% of
+ * the requested <code>maxPointsInLeafNode</code>. Values that fall exactly
+ * on a cell boundary may be in either cell.
+ *
+ * <p>The number of dimensions can be 1 to 255, but every byte[] value is fixed length.
+ *
+ * <p>
+ * See <a href="https://www.cs.duke.edu/~pankaj/publications/papers/bkd-sstd.pdf">this paper</a> for details.
+ *
+ * <p>This consumes heap during writing: it allocates a <code>LongBitSet(numPoints)</code>,
+ * and then uses up to the specified {@code maxMBSortInHeap} heap space for writing.
+ *
+ * <p>
+ * <b>NOTE</b>: This can write at most Integer.MAX_VALUE * <code>maxPointsInLeafNode</code> total points, and
+ *
+ * @lucene.experimental */
+
+public final class BKDWriter {
+
+ /** How many bytes each docs takes in the fixed-width offline format */
+ private final int bytesPerDoc;
+
+ static final boolean DEBUG = true;
+
+ public static final int DEFAULT_MAX_POINTS_IN_LEAF_NODE = 1024;
+
+ public static final float DEFAULT_MAX_MB_SORT_IN_HEAP = 16.0f;
+
+ private final byte[] scratchBytes;
+ private final ByteArrayDataOutput scratchBytesOutput;
+
+ /** How many dimensions we are indexing */
+ final int numDims;
+
+ /** How many bytes each value in each dimension takes. */
+ final int bytesPerDim;
+
+ /** numDims * bytesPerDim */
+ final int packedBytesLength;
+
+ final byte[] scratchMax;
+ final byte[] scratchDiff;
+ final byte[] scratchPackedValue;
+ final byte[] scratch1;
+ final byte[] scratch2;
+
+ // nocommit grep for lat / lon and fix!
+
+ private OfflineSorter.ByteSequencesWriter writer;
+ private HeapWriter heapWriter;
+
+ private Path tempInput;
+ private final int maxPointsInLeafNode;
+ private final int maxPointsSortInHeap;
+
+ private long pointCount;
+
+ public BKDWriter(int numDims, int bytesPerDim) throws IOException {
+ this(numDims, bytesPerDim, DEFAULT_MAX_POINTS_IN_LEAF_NODE, DEFAULT_MAX_MB_SORT_IN_HEAP);
+ }
+
+ public BKDWriter(int numDims, int bytesPerDim, int maxPointsInLeafNode, float maxMBSortInHeap) throws IOException {
+ verifyParams(numDims, maxPointsInLeafNode, maxMBSortInHeap);
+ this.maxPointsInLeafNode = maxPointsInLeafNode;
+ this.numDims = numDims;
+ this.bytesPerDim = bytesPerDim;
+ packedBytesLength = numDims * bytesPerDim;
+
+ scratchMax = new byte[bytesPerDim];
+ scratchDiff = new byte[bytesPerDim];
+ scratchPackedValue = new byte[packedBytesLength];
+ scratch1 = new byte[packedBytesLength];
+ scratch2 = new byte[packedBytesLength];
+
+ // dimensional values (numDims * bytesPerDim) + ord (long) + docID (int)
+ bytesPerDoc = packedBytesLength + RamUsageEstimator.NUM_BYTES_LONG + RamUsageEstimator.NUM_BYTES_INT;
+ scratchBytes = new byte[bytesPerDoc];
+ scratchBytesOutput = new ByteArrayDataOutput(scratchBytes);
+
+ // nocommit verify this is correct! see how much heap we actually use
+ // We must be able to hold at least the leaf node in heap at write:
+ maxPointsSortInHeap = Math.max(maxPointsInLeafNode, (int) (0.25 * (maxMBSortInHeap * 1024 * 1024) / bytesPerDoc));
+
+ // We write first maxPointsSortInHeap in heap, then cutover to offline for additional points:
+ heapWriter = new HeapWriter(16, maxPointsSortInHeap, packedBytesLength);
+ }
+
+ public static void verifyParams(int numDims, int maxPointsInLeafNode, float maxMBSortInHeap) {
+ // We encode dim in a single byte in the splitPackedValues, but we only expose 4 bits for it now, in case we want to use
+ // remaining 4 bits for another purpose later
+ if (numDims < 1 || numDims > 15) {
+ throw new IllegalArgumentException("numDims must be 1 .. 15 (got: " + numDims + ")");
+ }
+ if (maxPointsInLeafNode <= 0) {
+ throw new IllegalArgumentException("maxPointsInLeafNode must be > 0; got " + maxPointsInLeafNode);
+ }
+ if (maxPointsInLeafNode > ArrayUtil.MAX_ARRAY_LENGTH) {
+ throw new IllegalArgumentException("maxPointsInLeafNode must be <= ArrayUtil.MAX_ARRAY_LENGTH (= " + ArrayUtil.MAX_ARRAY_LENGTH + "); got " + maxPointsInLeafNode);
+ }
+ if (maxMBSortInHeap < 0.0) {
+ throw new IllegalArgumentException("maxMBSortInHeap must be >= 0.0 (got: " + maxMBSortInHeap + ")");
+ }
+ }
+
+ /** If the current segment has too many points then we switchover to temp files / offline sort. */
+ private void switchToOffline() throws IOException {
+ System.out.println("W: switchToOffline");
+
+ // For each .add we just append to this input file, then in .finish we sort this input and resursively build the tree:
+ tempInput = Files.createTempFile(OfflineSorter.getDefaultTempDir(), "in", "");
+ writer = new OfflineSorter.ByteSequencesWriter(tempInput);
+ Reader reader = heapWriter.getReader(0);
+ for(int i=0;i<pointCount;i++) {
+ boolean hasNext = reader.next();
+ assert hasNext;
+
+ byte[] packedValue = reader.packedValue();
+
+ scratchBytesOutput.reset(scratchBytes);
+ scratchBytesOutput.writeBytes(packedValue, 0, packedValue.length);
+ scratchBytesOutput.writeVInt(heapWriter.docIDs[i]);
+ scratchBytesOutput.writeVLong(i);
+ // TODO: can/should OfflineSorter optimize the fixed-width case?
+ writer.write(scratchBytes, 0, scratchBytes.length);
+ }
+
+ heapWriter = null;
+ }
+
+ public void add(byte[] packedValue, int docID) throws IOException {
+ if (packedValue.length != packedBytesLength) {
+ throw new IllegalArgumentException("packedValue should be length=" + packedBytesLength + " (got: " + packedValue.length + ")");
+ }
+
+ if (pointCount >= maxPointsSortInHeap) {
+ if (writer == null) {
+ switchToOffline();
+ }
+ scratchBytesOutput.reset(scratchBytes);
+ scratchBytesOutput.writeBytes(packedValue, 0, packedValue.length);
+ scratchBytesOutput.writeVInt(docID);
+ scratchBytesOutput.writeVLong(pointCount);
+ writer.write(scratchBytes, 0, scratchBytes.length);
+ } else {
+ // Not too many points added yet, continue using heap:
+ heapWriter.append(packedValue, pointCount, docID);
+ }
+
+ pointCount++;
+ }
+
+ // TODO: if we fixed each partition step to just record the file offset at the "split point", we could probably handle variable length
+ // encoding?
+
+ /** Changes incoming {@link ByteSequencesWriter} file to to fixed-width-per-entry file, because we need to be able to slice
+ * as we recurse in {@link #build}. */
+ private Writer convertToFixedWidth(Path in) throws IOException {
+ BytesRefBuilder scratch = new BytesRefBuilder();
+ scratch.grow(bytesPerDoc);
+ BytesRef bytes = scratch.get();
+ ByteArrayDataInput dataReader = new ByteArrayDataInput();
+
+ OfflineSorter.ByteSequencesReader reader = null;
+ Writer sortedWriter = null;
+ boolean success = false;
+ byte[] packedValue = new byte[packedBytesLength];
+ try {
+ reader = new OfflineSorter.ByteSequencesReader(in);
+ sortedWriter = getWriter(pointCount);
+ for (long i=0;i<pointCount;i++) {
+ boolean result = reader.read(scratch);
+ assert result;
+ dataReader.reset(bytes.bytes, bytes.offset, bytes.length);
+ dataReader.readBytes(packedValue, 0, packedValue.length);
+ int docID = dataReader.readVInt();
+ long ord = dataReader.readVLong();
+ assert docID >= 0: "docID=" + docID;
+ sortedWriter.append(packedValue, ord, docID);
+ }
+ success = true;
+ } finally {
+ if (success) {
+ IOUtils.close(sortedWriter, reader);
+ } else {
+ IOUtils.closeWhileHandlingException(sortedWriter, reader);
+ try {
+ sortedWriter.destroy();
+ } catch (Throwable t) {
+ // Suppress to keep throwing original exc
+ }
+ }
+ }
+
+ return sortedWriter;
+ }
+
+ private int compare(byte[] a, byte[] b, int dim) {
+ int start = dim * bytesPerDim;
+ int end = start + bytesPerDim;
+ for(int i=start;i<end;i++) {
+ int diff = (a[i]&0xff) - (b[i]&0xff);
+ if (diff != 0) {
+ return diff;
+ }
+ }
+
+ return 0;
+ }
+
+ /** If dim=-1 we sort by docID, else by that dim. */
+ private void sortHeapWriter(final HeapWriter writer, int start, int length, int dim) {
+
+ assert pointCount < Integer.MAX_VALUE;
+
+ // All buffered points are still in heap; just do in-place sort:
+ new InPlaceMergeSorter() {
+ @Override
+ protected void swap(int i, int j) {
+ int docID = writer.docIDs[i];
+ writer.docIDs[i] = writer.docIDs[j];
+ writer.docIDs[j] = docID;
+
+ long ord = writer.ords[i];
+ writer.ords[i] = writer.ords[j];
+ writer.ords[j] = ord;
+
+ // scratch1 = values[i]
+ writer.readPackedValue(i, scratch1);
+ // scratch2 = values[j]
+ writer.readPackedValue(j, scratch2);
+ // values[i] = scratch2
+ writer.writePackedValue(i, scratch2);
+ // values[j] = scratch1
+ writer.writePackedValue(j, scratch1);
+ }
+
+ @Override
+ protected int compare(int i, int j) {
+ if (dim != -1) {
+ writer.readPackedValue(i, scratch1);
+ writer.readPackedValue(j, scratch2);
+ int cmp = Util.compare(bytesPerDim, scratch1, dim, scratch2, dim);
+ if (cmp != 0) {
+ return cmp;
+ }
+ }
+
+ // Tie-break
+ int cmp = Integer.compare(writer.docIDs[i], writer.docIDs[j]);
+ if (cmp != 0) {
+ return cmp;
+ }
+
+ return Long.compare(writer.ords[i], writer.ords[j]);
+ }
+ }.sort(start, start+length);
+ }
+
+ private Writer sort(int dim) throws IOException {
+
+ if (heapWriter != null) {
+
+ sortHeapWriter(heapWriter, 0, (int) pointCount, dim);
+
+ // nocommit shrink wrap?
+ heapWriter.close();
+ return heapWriter;
+ } else {
+
+ // Offline sort:
+ assert tempInput != null;
+
+ final ByteArrayDataInput reader = new ByteArrayDataInput();
+ Comparator<BytesRef> cmp = new Comparator<BytesRef>() {
+ private final ByteArrayDataInput readerB = new ByteArrayDataInput();
+
+ @Override
+ public int compare(BytesRef a, BytesRef b) {
+ reader.reset(a.bytes, a.offset, a.length);
+ reader.readBytes(scratch1, 0, scratch1.length);
+ final int docIDA = reader.readVInt();
+ final long ordA = reader.readVLong();
+
+ reader.reset(b.bytes, b.offset, b.length);
+ reader.readBytes(scratch2, 0, scratch2.length);
+ final int docIDB = reader.readVInt();
+ final long ordB = reader.readVLong();
+
+ int cmp = Util.compare(bytesPerDim, scratch1, dim, scratch2, dim);
+
+ if (cmp != 0) {
+ return cmp;
+ }
+
+ // Tie-break
+ cmp = Integer.compare(docIDA, docIDB);
+ if (cmp != 0) {
+ return cmp;
+ }
+
+ return Long.compare(ordA, ordB);
+ }
+ };
+
+ Path sorted = Files.createTempFile(OfflineSorter.getDefaultTempDir(), "sorted", "");
+ boolean success = false;
+
+ try {
+ OfflineSorter sorter = new OfflineSorter(cmp);
+ sorter.sort(tempInput, sorted);
+ Writer writer = convertToFixedWidth(sorted);
+ success = true;
+ return writer;
+ } finally {
+ if (success) {
+ IOUtils.rm(sorted);
+ } else {
+ IOUtils.deleteFilesIgnoringExceptions(sorted);
+ }
+ }
+ }
+ }
+
+ /** Writes the BKD tree to the provided {@link IndexOutput} and returns the file offset where index was written. */
+ public long finish(IndexOutput out) throws IOException {
+ //System.out.println("\nBKDTreeWriter.finish pointCount=" + pointCount + " out=" + out + " heapWriter=" + heapWriter);
+
+ if (writer != null) {
+ writer.close();
+ }
+
+ LongBitSet ordBitSet = new LongBitSet(pointCount);
+
+ long countPerLeaf = pointCount;
+ long innerNodeCount = 1;
+
+ while (countPerLeaf > maxPointsInLeafNode) {
+ countPerLeaf = (countPerLeaf+1)/2;
+ innerNodeCount *= 2;
+ }
+
+ //System.out.println("innerNodeCount=" + innerNodeCount);
+
+ if (1+2*innerNodeCount >= Integer.MAX_VALUE) {
+ throw new IllegalStateException("too many nodes; increase maxPointsInLeafNode (currently " + maxPointsInLeafNode + ") and reindex");
+ }
+
+ innerNodeCount--;
+
+ int numLeaves = (int) (innerNodeCount+1);
+
+ // NOTE: we could save the 1+ here, to use a bit less heap at search time, but then we'd need a somewhat costly check at each
+ // step of the recursion to recompute the split dim:
+
+ // Indexed by nodeID, but first (root) nodeID is 1. We do 1+ because the lead byte at each recursion says which dim we split on.
+ byte[] splitPackedValues = new byte[Math.toIntExact(numLeaves*(1+bytesPerDim))];
+
+ // +1 because leaf count is power of 2 (e.g. 8), and innerNodeCount is power of 2 minus 1 (e.g. 7)
+ long[] leafBlockFPs = new long[numLeaves];
+
+ // Make sure the math above "worked":
+ assert pointCount / numLeaves <= maxPointsInLeafNode: "pointCount=" + pointCount + " numLeaves=" + numLeaves + " maxPointsInLeafNode=" + maxPointsInLeafNode;
+
+ // Sort all docs once by each dimension:
+ PathSlice[] sortedWriters = new PathSlice[numDims];
+
+ boolean success = false;
+ try {
+ for(int dim=0;dim<numDims;dim++) {
+ sortedWriters[dim] = new PathSlice(sort(dim), 0, pointCount);
+ }
+ heapWriter = null;
+
+ byte[] minPacked = new byte[packedBytesLength];
+ byte[] maxPacked = new byte[packedBytesLength];
+ Arrays.fill(maxPacked, (byte) 0xff);
+
+ build(1, numLeaves, sortedWriters,
+ ordBitSet, out,
+ minPacked, maxPacked,
+ splitPackedValues,
+ leafBlockFPs);
+ success = true;
+ } finally {
+ if (success) {
+ for(int dim=0;dim<numDims;dim++) {
+ sortedWriters[dim].writer.destroy();
+ }
+ IOUtils.rm(tempInput);
+ } else {
+ for(int dim=0;dim<numDims;dim++) {
+ try {
+ sortedWriters[dim].writer.destroy();
+ } catch (Throwable t) {
+ // Suppress to keep throwing original exc
+ }
+ }
+ IOUtils.deleteFilesIgnoringExceptions(tempInput);
+ }
+ }
+
+ //System.out.println("Total nodes: " + innerNodeCount);
+
+ // nocommit write header + version here & read/confirm in BKDReader
+
+ // Write index:
+ long indexFP = out.getFilePointer();
+ out.writeVInt(numDims);
+ out.writeVInt(maxPointsInLeafNode);
+ out.writeVInt(bytesPerDim);
+
+ out.writeVInt(numLeaves);
+
+ // NOTE: splitPackedValues[0] is unused, because nodeID is 1-based:
+ out.writeBytes(splitPackedValues, 0, splitPackedValues.length);
+
+ for (int i=0;i<leafBlockFPs.length;i++) {
+ out.writeVLong(leafBlockFPs[i]);
+ }
+
+ return indexFP;
+ }
+
+ /** Sliced reference to points in an OfflineSorter.ByteSequencesWriter file. */
+ private static final class PathSlice {
+ final Writer writer;
+ final long start;
+ final long count;
+
+ public PathSlice(Writer writer, long start, long count) {
+ this.writer = writer;
+ this.start = start;
+ this.count = count;
+ }
+
+ @Override
+ public String toString() {
+ return "PathSlice(start=" + start + " count=" + count + " writer=" + writer + ")";
+ }
+ }
+
+ /** Marks bits for the ords (points) that belong in the right sub tree (those docs that have values >= the splitValue). */
+ private byte[] markRightTree(long rightCount, int splitDim, PathSlice source, LongBitSet ordBitSet) throws IOException {
+
+ // nocommit instead of partitioning to disk can't we just alloc new bitsets and pass those down?
+
+ // Now we mark ords that fall into the right half, so we can partition on all other dims that are not the split dim:
+ assert ordBitSet.cardinality() == 0: "cardinality=" + ordBitSet.cardinality();
+
+ // Read the split value: just open a reader, seek'd to the next value after leftCount, then read its value:
+ Reader reader = source.writer.getReader(source.start + source.count - rightCount);
+ boolean success = false;
+ try {
+ boolean result = reader.next();
+ assert result;
+
+ System.arraycopy(reader.packedValue(), splitDim*bytesPerDim, scratchMax, 0, bytesPerDim);
+
+ ordBitSet.set(reader.ord());
+
+ // Start at 1 because we already did the first value above (so we could keep the split value):
+ for(int i=1;i<rightCount;i++) {
+ result = reader.next();
+ assert result;
+ ordBitSet.set(reader.ord());
+ }
+
+ success = true;
+ } finally {
+ if (success) {
+ IOUtils.close(reader);
+ } else {
+ IOUtils.closeWhileHandlingException(reader);
+ }
+ }
+
+ assert rightCount == ordBitSet.cardinality(): "rightCount=" + rightCount + " cardinality=" + ordBitSet.cardinality();
+
+ // nocommit rename scratchMax
+ return scratchMax;
+ }
+
+ /** Called only in assert */
+ private boolean valueInBounds(byte[] packedValue, byte[] minPackedValue, byte[] maxPackedValue) {
+ for(int dim=0;dim<numDims;dim++) {
+ if (Util.compare(bytesPerDim, packedValue, dim, minPackedValue, dim) < 0) {
+ return false;
+ }
+ if (Util.compare(bytesPerDim, packedValue, dim, maxPackedValue, dim) > 0) {
+ return false;
+ }
+ }
+
+ return true;
+ }
+
+ /** The array (sized numDims) of PathSlice describe the cell we have currently recursed to. */
+ private void build(int nodeID, int leafNodeOffset,
+ PathSlice[] slices,
+ LongBitSet ordBitSet,
+ IndexOutput out,
+ byte[] minPackedValue, byte[] maxPackedValue,
+ byte[] splitPackedValues,
+ long[] leafBlockFPs) throws IOException {
+
+ // Find which dim has the largest span so we can split on it:
+ int splitDim = -1;
+ for(int dim=0;dim<numDims;dim++) {
+ Util.subtract(bytesPerDim, dim, maxPackedValue, minPackedValue, scratchDiff);
+ if (splitDim == -1 || Util.compare(bytesPerDim, scratchDiff, 0, scratchMax, 0) > 0) {
+ System.arraycopy(scratchDiff, 0, scratchMax, 0, bytesPerDim);
+ splitDim = dim;
+ }
+
+ assert slices[dim].count == slices[0].count;
+ }
+
+ PathSlice source = slices[splitDim];
+
+ if (DEBUG) System.out.println("\nBUILD: nodeID=" + nodeID + " leafNodeOffset=" + leafNodeOffset + " splitDim=" + splitDim + "\n count=" + source.count + "\n source=" + source +
+ "\n min=" + bytesToInt(minPackedValue, 0) + " max=" + bytesToInt(maxPackedValue, 0));
+
+ if (nodeID >= leafNodeOffset) {
+ // Leaf node: write block
+ if (DEBUG) System.out.println(" leaf");
+
+ // We ensured that maxPointsSortInHeap was >= maxPointsInLeafNode, so we better be in heap at this point:
+ assert source.writer instanceof HeapWriter;
+ HeapWriter heapSource = (HeapWriter) source.writer;
+
+ // Sort by docID in the leaf so we can delta-vInt encode:
+ sortHeapWriter(heapSource, Math.toIntExact(source.start), Math.toIntExact(source.count), -1);
+
+ int lastDocID = 0;
+
+ // Save the block file pointer:
+ leafBlockFPs[nodeID - leafNodeOffset] = out.getFilePointer();
+
+ out.writeVInt(Math.toIntExact(source.count));
+
+ // Write docIDs first, as their own chunk, so that at intersect time we can add all docIDs w/o
+ // loading the values:
+ for (int i=0;i<source.count;i++) {
+ int docID = heapSource.docIDs[Math.toIntExact(source.start + i)];
+ System.out.println(" docID=" + docID);
+ out.writeVInt(docID - lastDocID);
+ lastDocID = docID;
+ }
+
+ // TODO: we should delta compress / only write suffix bytes, like terms dict (the values will all be "close together" since we are at
+ // a leaf cell):
+
+ // Now write the full values:
+ for (int i=0;i<source.count;i++) {
+ // TODO: we could do bulk copying here, avoiding the intermediate copy:
+ heapSource.readPackedValue(Math.toIntExact(source.start + i), scratchPackedValue);
+
+ // Make sure this value does in fact fall within this leaf cell:
+ assert valueInBounds(scratchPackedValue, minPackedValue, maxPackedValue);
+ out.writeBytes(scratchPackedValue, 0, scratchPackedValue.length);
+ }
+
+ } else {
+ // Inner node: partition/recurse
+
+ assert nodeID < splitPackedValues.length: "nodeID=" + nodeID + " splitValues.length=" + splitPackedValues.length;
+
+ // How many points will be in the left tree:
+ long rightCount = source.count / 2;
+ long leftCount = source.count - rightCount;
+
+ byte[] splitValue = markRightTree(rightCount, splitDim, source, ordBitSet);
+ int address = nodeID * (1+bytesPerDim);
+ splitPackedValues[address] = (byte) splitDim;
+ System.arraycopy(splitValue, 0, splitPackedValues, address + 1, bytesPerDim);
+
+ // Partition all PathSlice that are not the split dim into sorted left and right sets, so we can recurse:
+
+ PathSlice[] leftSlices = new PathSlice[numDims];
+ PathSlice[] rightSlices = new PathSlice[numDims];
+
+ byte[] minSplitPackedValue = new byte[packedBytesLength];
+ System.arraycopy(minPackedValue, 0, minSplitPackedValue, 0, packedBytesLength);
+
+ byte[] maxSplitPackedValue = new byte[packedBytesLength];
+ System.arraycopy(maxPackedValue, 0, maxSplitPackedValue, 0, packedBytesLength);
+
+ for(int dim=0;dim<numDims;dim++) {
+ if (dim == splitDim) {
+ // No need to partition on this dim since it's a simple slice of the incoming already sorted slice.
+ leftSlices[dim] = new PathSlice(source.writer, source.start, leftCount);
+ rightSlices[dim] = new PathSlice(source.writer, source.start + leftCount, rightCount);
+ System.arraycopy(splitValue, 0, minSplitPackedValue, dim*bytesPerDim, bytesPerDim);
+ System.arraycopy(splitValue, 0, maxSplitPackedValue, dim*bytesPerDim, bytesPerDim);
+ continue;
+ }
+
+ Writer leftWriter = null;
+ Writer rightWriter = null;
+ Reader reader = null;
+
+ boolean success = false;
+
+ int nextRightCount = 0;
+
+ try {
+ leftWriter = getWriter(leftCount);
+ rightWriter = getWriter(source.count - leftCount);
+
+ //if (DEBUG) System.out.println(" partition:\n splitValueEnc=" + splitValue + "\n " + nextSource + "\n --> leftSorted=" + leftWriter + "\n --> rightSorted=" + rightWriter + ")");
+ reader = slices[dim].writer.getReader(slices[dim].start);
+
+ // Partition this source according to how the splitDim split the values:
+ for (int i=0;i<source.count;i++) {
+ boolean result = reader.next();
+ assert result;
+ byte[] packedValue = reader.packedValue();
+ long ord = reader.ord();
+ int docID = reader.docID();
+ if (ordBitSet.get(ord)) {
+ rightWriter.append(packedValue, ord, docID);
+ nextRightCount++;
+ } else {
+ leftWriter.append(packedValue, ord, docID);
+ }
+ }
+ success = true;
+ } finally {
+ if (success) {
+ IOUtils.close(reader, leftWriter, rightWriter);
+ } else {
+ IOUtils.closeWhileHandlingException(reader, leftWriter, rightWriter);
+ // nocommit we must also destroy all prior writers here (so temp files are removed)
+ }
+ }
+
+ leftSlices[dim] = new PathSlice(leftWriter, 0, leftCount);
+ rightSlices[dim] = new PathSlice(rightWriter, 0, rightCount);
+
+ assert rightCount == nextRightCount: "rightCount=" + rightCount + " nextRightCount=" + nextRightCount;
+ }
+
+ System.out.println("pointCount=" + pointCount);
+ ordBitSet.clear(0, pointCount);
+
+ boolean success = false;
+ try {
+ // Recurse on left tree:
+ build(2*nodeID, leafNodeOffset, leftSlices,
+ ordBitSet, out,
+ minPackedValue, maxSplitPackedValue,
+ splitPackedValues, leafBlockFPs);
+ for(int dim=0;dim<numDims;dim++) {
+ if (dim != splitDim) {
+ // nocommit need try/finally monster around this?
+ leftSlices[dim].writer.destroy();
+ }
+ }
+
+ // Recurse on right tree:
+ build(2*nodeID+1, leafNodeOffset, rightSlices,
+ ordBitSet, out,
+ minSplitPackedValue, maxPackedValue,
+ splitPackedValues, leafBlockFPs);
+ for(int dim=0;dim<numDims;dim++) {
+ if (dim != splitDim) {
+ // nocommit need try/finally monster around this?
+ rightSlices[dim].writer.destroy();
+ }
+ }
+ success = true;
+ } finally {
+ if (success == false) {
+ for(int dim=0;dim<numDims;dim++) {
+ if (dim != splitDim) {
+ try {
+ leftSlices[dim].writer.destroy();
+ } catch (Throwable t) {
+ // Suppress to keep throwing original exc
+ }
+ try {
+ rightSlices[dim].writer.destroy();
+ } catch (Throwable t) {
+ // Suppress to keep throwing original exc
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+
+ Writer getWriter(long count) throws IOException {
+ System.out.println("W: getWriter count=" + count);
+ if (count <= maxPointsSortInHeap) {
+ System.out.println(" heap");
+ int size = Math.toIntExact(count);
+ return new HeapWriter(size, size, packedBytesLength);
+ } else {
+ System.out.println(" offline");
+ return new OfflineWriter(count, packedBytesLength);
+ }
+ }
+
+ // nocommit removeme
+ static int bytesToInt(byte[] src, int index) {
+ int x = 0;
+ for(int i=0;i<4;i++) {
+ x |= (src[4*index+i] & 0xff) << (24-i*8);
+ }
+ // Re-flip the sign bit to restore the original value:
+ return x ^ 0x80000000;
+ }
+}
Added: lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/HeapReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/HeapReader.java?rev=1707203&view=auto
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/HeapReader.java (added)
+++ lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/HeapReader.java Wed Oct 7 08:34:30 2015
@@ -0,0 +1,86 @@
+package org.apache.lucene.util.bkd;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.List;
+
+import org.apache.lucene.util.PagedBytes;
+
+final class HeapReader implements Reader {
+ private int curRead;
+ final List<byte[]> blocks;
+ final int valuesPerBlock;
+ final int packedBytesLength;
+ final long[] ords;
+ final int[] docIDs;
+ final int end;
+ final byte[] scratch;
+
+ HeapReader(List<byte[]> blocks, int valuesPerBlock, int packedBytesLength, long[] ords, int[] docIDs, int start, int end) {
+ this.blocks = blocks;
+ this.valuesPerBlock = valuesPerBlock;
+ this.ords = ords;
+ this.docIDs = docIDs;
+ curRead = start-1;
+ this.end = end;
+ this.packedBytesLength = packedBytesLength;
+ scratch = new byte[packedBytesLength];
+ }
+
+ void writePackedValue(int index, byte[] bytes) {
+ int block = index / valuesPerBlock;
+ int blockIndex = index % valuesPerBlock;
+ while (blocks.size() <= block) {
+ blocks.add(new byte[valuesPerBlock*packedBytesLength]);
+ }
+ System.arraycopy(bytes, 0, blocks.get(blockIndex), blockIndex * packedBytesLength, packedBytesLength);
+ }
+
+ void readPackedValue(int index, byte[] bytes) {
+ int block = index / valuesPerBlock;
+ int blockIndex = index % valuesPerBlock;
+ System.arraycopy(blocks.get(block), blockIndex * packedBytesLength, bytes, 0, packedBytesLength);
+ }
+
+ @Override
+ public boolean next() {
+ curRead++;
+ return curRead < end;
+ }
+
+ @Override
+ public byte[] packedValue() {
+ readPackedValue(curRead, scratch);
+ return scratch;
+ }
+
+ @Override
+ public int docID() {
+ return docIDs[curRead];
+ }
+
+ @Override
+ public long ord() {
+ return ords[curRead];
+ }
+
+ @Override
+ public void close() {
+ }
+}
+
Added: lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/HeapWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/HeapWriter.java?rev=1707203&view=auto
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/HeapWriter.java (added)
+++ lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/HeapWriter.java Wed Oct 7 08:34:30 2015
@@ -0,0 +1,114 @@
+package org.apache.lucene.util.bkd;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
+
+final class HeapWriter implements Writer {
+ int[] docIDs;
+ long[] ords;
+ private int nextWrite;
+ private boolean closed;
+ final int maxSize;
+ final int valuesPerBlock;
+ final int packedBytesLength;
+ final List<byte[]> blocks = new ArrayList<>();
+
+ public HeapWriter(int initSize, int maxSize, int packedBytesLength) {
+ System.out.println("HeapWriter.init initSize=" + initSize + " maxSize=" + maxSize + " packedBytesLength=" + packedBytesLength);
+ docIDs = new int[initSize];
+ ords = new long[initSize];
+ this.maxSize = maxSize;
+ this.packedBytesLength = packedBytesLength;
+ // 4K per page, unless each value is > 4K:
+ valuesPerBlock = Math.max(1, 4096/packedBytesLength);
+ }
+
+ void readPackedValue(int index, byte[] bytes) {
+ assert bytes.length == packedBytesLength;
+ int block = index / valuesPerBlock;
+ int blockIndex = index % valuesPerBlock;
+ System.arraycopy(blocks.get(block), blockIndex * packedBytesLength, bytes, 0, packedBytesLength);
+ }
+
+ void writePackedValue(int index, byte[] bytes) {
+ assert bytes.length == packedBytesLength;
+ int block = index / valuesPerBlock;
+ int blockIndex = index % valuesPerBlock;
+ //System.out.println("writePackedValue: index=" + index + " bytes.length=" + bytes.length + " block=" + block + " blockIndex=" + blockIndex + " valuesPerBlock=" + valuesPerBlock);
+ while (blocks.size() <= block) {
+ // If this is the last block, only allocate as large as necessary for maxSize:
+ int valuesInBlock = Math.min(valuesPerBlock, maxSize - (blocks.size() * valuesPerBlock));
+ blocks.add(new byte[valuesInBlock*packedBytesLength]);
+ }
+ System.arraycopy(bytes, 0, blocks.get(block), blockIndex * packedBytesLength, packedBytesLength);
+ }
+
+ private int[] growExact(int[] arr, int size) {
+ assert size > arr.length;
+ int[] newArr = new int[size];
+ System.arraycopy(arr, 0, newArr, 0, arr.length);
+ return newArr;
+ }
+
+ private long[] growExact(long[] arr, int size) {
+ assert size > arr.length;
+ long[] newArr = new long[size];
+ System.arraycopy(arr, 0, newArr, 0, arr.length);
+ return newArr;
+ }
+
+ @Override
+ public void append(byte[] packedValue, long ord, int docID) {
+ assert closed == false;
+ assert packedValue.length == packedBytesLength;
+ if (ords.length == nextWrite) {
+ int nextSize = Math.min(maxSize, ArrayUtil.oversize(nextWrite+1, RamUsageEstimator.NUM_BYTES_INT));
+ assert nextSize > nextWrite: "nextSize=" + nextSize + " vs nextWrite=" + nextWrite;
+ ords = growExact(ords, nextSize);
+ docIDs = growExact(docIDs, nextSize);
+ }
+ writePackedValue(nextWrite, packedValue);
+ ords[nextWrite] = ord;
+ docIDs[nextWrite] = docID;
+ nextWrite++;
+ }
+
+ @Override
+ public Reader getReader(long start) {
+ return new HeapReader(blocks, valuesPerBlock, packedBytesLength, ords, docIDs, (int) start, nextWrite);
+ }
+
+ @Override
+ public void close() {
+ closed = true;
+ }
+
+ @Override
+ public void destroy() {
+ }
+
+ @Override
+ public String toString() {
+ return "HeapWriter(count=" + nextWrite + " alloc=" + ords.length + ")";
+ }
+}
Added: lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/OfflineReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/OfflineReader.java?rev=1707203&view=auto
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/OfflineReader.java (added)
+++ lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/OfflineReader.java Wed Oct 7 08:34:30 2015
@@ -0,0 +1,87 @@
+package org.apache.lucene.util.bkd;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.BufferedInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+
+import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.util.RamUsageEstimator;
+
+final class OfflineReader implements Reader {
+ final InputStreamDataInput in;
+ long countLeft;
+ private final byte[] packedValue;
+ private long ord;
+ private int docID;
+ final int bytesPerDoc;
+
+ OfflineReader(Path tempFile, int packedBytesLength, long start, long length) throws IOException {
+ bytesPerDoc = packedBytesLength + RamUsageEstimator.NUM_BYTES_LONG + RamUsageEstimator.NUM_BYTES_INT;
+ long seekFP = start * bytesPerDoc;
+ // nocommit cutover to Directory API
+ InputStream fis = Files.newInputStream(tempFile);
+ long skipped = 0;
+ while (skipped < seekFP) {
+ long inc = fis.skip(seekFP - skipped);
+ skipped += inc;
+ if (inc == 0) {
+ throw new RuntimeException("skip returned 0");
+ }
+ }
+ in = new InputStreamDataInput(new BufferedInputStream(fis));
+ this.countLeft = length;
+ packedValue = new byte[packedBytesLength];
+ }
+
+ @Override
+ public boolean next() throws IOException {
+ if (countLeft == 0) {
+ return false;
+ }
+ countLeft--;
+ in.readBytes(packedValue, 0, packedValue.length);
+ ord = in.readLong();
+ docID = in.readInt();
+ return true;
+ }
+
+ @Override
+ public byte[] packedValue() {
+ return packedValue;
+ }
+
+ @Override
+ public long ord() {
+ return ord;
+ }
+
+ @Override
+ public int docID() {
+ return docID;
+ }
+
+ @Override
+ public void close() throws IOException {
+ in.close();
+ }
+}
+
Added: lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/OfflineWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/OfflineWriter.java?rev=1707203&view=auto
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/OfflineWriter.java (added)
+++ lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/OfflineWriter.java Wed Oct 7 08:34:30 2015
@@ -0,0 +1,84 @@
+package org.apache.lucene.util.bkd;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.BufferedOutputStream;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+
+import org.apache.lucene.store.ByteArrayDataOutput;
+import org.apache.lucene.store.OutputStreamDataOutput;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.OfflineSorter;
+import org.apache.lucene.util.RamUsageEstimator;
+
+final class OfflineWriter implements Writer {
+
+ final Path tempFile;
+ final OutputStreamDataOutput out;
+ final long count;
+ final int packedBytesLength;
+ final int bytesPerDoc;
+ private long countWritten;
+ private boolean closed;
+
+ public OfflineWriter(long count, int packedBytesLength) throws IOException {
+ tempFile = Files.createTempFile(OfflineSorter.getDefaultTempDir(), "size" + count + ".", "");
+ // nocommit cutover to Directory API
+ out = new OutputStreamDataOutput(new BufferedOutputStream(Files.newOutputStream(tempFile)));
+ this.packedBytesLength = packedBytesLength;
+ bytesPerDoc = packedBytesLength + RamUsageEstimator.NUM_BYTES_LONG + RamUsageEstimator.NUM_BYTES_INT;
+ this.count = count;
+ }
+
+ @Override
+ public void append(byte[] packedValue, long ord, int docID) throws IOException {
+ assert packedValue.length == packedBytesLength;
+ out.writeBytes(packedValue, 0, packedValue.length);
+ out.writeLong(ord);
+ out.writeInt(docID);
+ countWritten++;
+ }
+
+ @Override
+ public Reader getReader(long start) throws IOException {
+ assert closed;
+ return new OfflineReader(tempFile, packedBytesLength, start, count-start);
+ }
+
+ @Override
+ public void close() throws IOException {
+ closed = true;
+ out.close();
+ if (count != countWritten) {
+ throw new IllegalStateException("wrote " + countWritten + " values, but expected " + count);
+ }
+ }
+
+ @Override
+ public void destroy() throws IOException {
+ IOUtils.rm(tempFile);
+ }
+
+ @Override
+ public String toString() {
+ return "OfflineWriter(count=" + count + " tempFile=" + tempFile + ")";
+ }
+}
+
Added: lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/Reader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/Reader.java?rev=1707203&view=auto
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/Reader.java (added)
+++ lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/Reader.java Wed Oct 7 08:34:30 2015
@@ -0,0 +1,31 @@
+package org.apache.lucene.util.bkd;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/** Abstracts away whether OfflineSorter or simple arrays in heap are used. */
+interface Reader extends Closeable {
+ boolean next() throws IOException;
+ // nocommit maybe caller should pass in reused buffer? reduces GC on recursion...?
+ byte[] packedValue();
+ long ord();
+ int docID();
+}
+
Added: lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/Util.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/Util.java?rev=1707203&view=auto
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/Util.java (added)
+++ lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/Util.java Wed Oct 7 08:34:30 2015
@@ -0,0 +1,71 @@
+package org.apache.lucene.util.bkd;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/** Utility methods to handle N-dimensional packed byte[] as if they were numbers! */
+final class Util {
+
+ private Util() {
+ // No instance
+ }
+
+ /** result = a - b */
+ public static void subtract(int bytesPerDim, int dim, byte[] a, byte[] b, byte[] result) {
+ int start = dim * bytesPerDim;
+ int end = start + bytesPerDim;
+ int carry = 0;
+ for(int i=end-1;i>=start;i--) {
+ int diff = (a[i]&0xff) - (b[i]&0xff) - carry;
+ if (diff < 0) {
+ diff += 255;
+ carry = 1;
+ }
+ result[i-start] = (byte) diff;
+ }
+ assert carry == 0;
+ }
+
+ /** Returns positive int if a > b, negative int if a < b and 0 if a == b */
+ public static int compare(int bytesPerDim, byte[] a, int aIndex, byte[] b, int bIndex) {
+ for(int i=0;i<bytesPerDim;i++) {
+ int cmp = (a[aIndex*bytesPerDim+i]&0xff) - (b[bIndex*bytesPerDim+i]&0xff);
+ if (cmp != 0) {
+ return cmp;
+ }
+ }
+
+ return 0;
+ }
+
+ /** Returns true if N-dim rect A contains N-dim rect B */
+ public static boolean contains(int bytesPerDim,
+ byte[] minPackedA, byte[] maxPackedA,
+ byte[] minPackedB, byte[] maxPackedB) {
+ int dims = minPackedA.length / bytesPerDim;
+ for(int dim=0;dim<dims;dim++) {
+ if (compare(bytesPerDim, minPackedA, dim, minPackedB, dim) > 0) {
+ return false;
+ }
+ if (compare(bytesPerDim, maxPackedA, dim, maxPackedB, dim) < 0) {
+ return false;
+ }
+ }
+
+ return true;
+ }
+}
Added: lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/Writer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/Writer.java?rev=1707203&view=auto
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/Writer.java (added)
+++ lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/Writer.java Wed Oct 7 08:34:30 2015
@@ -0,0 +1,29 @@
+package org.apache.lucene.util.bkd;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/** Abstracts away whether OfflineSorter or simple arrays in heap are used. */
+interface Writer extends Closeable {
+ void append(byte[] packedValue, long ord, int docID) throws IOException;
+ Reader getReader(long startDoc) throws IOException;
+ void destroy() throws IOException;
+}
+
Added: lucene/dev/branches/lucene6825/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKD.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKD.java?rev=1707203&view=auto
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKD.java (added)
+++ lucene/dev/branches/lucene6825/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKD.java Wed Oct 7 08:34:30 2015
@@ -0,0 +1,304 @@
+package org.apache.lucene.util.bkd;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.Arrays;
+import java.util.BitSet;
+
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+@SuppressSysoutChecks(bugUrl = "Stuff gets printed.")
+public class TestBKD extends LuceneTestCase {
+
+ // nocommit accum to long then cast to int?
+ static void intToBytes(int x, byte[] dest, int index) {
+ // Flip the sign bit, so negative ints sort before positive ints correctly:
+ x ^= 0x80000000;
+ for(int i=0;i<4;i++) {
+ dest[4*index+i] = (byte) (x >> 24-i*8);
+ }
+ }
+
+ // nocommit accum to long then cast to int?
+ static int bytesToInt(byte[] src, int index) {
+ int x = 0;
+ for(int i=0;i<4;i++) {
+ x |= (src[4*index+i] & 0xff) << (24-i*8);
+ }
+ // Re-flip the sign bit to restore the original value:
+ return x ^ 0x80000000;
+ }
+
+ /*
+ private static abstract class VerifyHits {
+ public abstract boolean accept(int docID, byte[] packedValues);
+ public abstract BKDReader.relation compare(byte[] minPacked, byte[] maxPacked);
+ }
+
+ private void verify(List<byte[]> values, VerifyHits verify) throws Exception {
+ int bytesPerDim = values.get(0).length;
+ try (Directory dir = newDirectory()) {
+ BKDWriter w = new BKDWriter(1, 4, 2, 1.0f);
+ byte[] scratch = new byte[4];
+ for(int docID=0;docID<values.size();docID++) {
+ w.add(values.get(docID), docID);
+ }
+
+ long indexFP;
+ try (IndexOutput out = dir.createOutput("bkd", IOContext.DEFAULT)) {
+ indexFP = w.finish(out);
+ }
+
+ try (IndexInput in = dir.openInput("bkd", IOContext.DEFAULT)) {
+ in.seek(indexFP);
+ BKDReader r = new BKDReader(in, 100);
+
+ // Simple 1D range query:
+ final int queryMin = 42;
+ final int queryMax = 87;
+
+ final BitSet hits = new BitSet();
+ r.intersect(new BKDReader.IntersectVisitor() {
+ @Override
+ public void visit(int docID) {
+ hits.set(docID);
+ System.out.println("visit docID=" + docID);
+ }
+
+ @Override
+ public void visit(int docID, byte[] packedValue) {
+ int x = bytesToInt(packedValue, 0);
+ System.out.println("visit docID=" + docID + " x=" + x);
+ if (x >= queryMin && x <= queryMax) {
+ hits.set(docID);
+ }
+ }
+
+ @Override
+ public BKDReader.Relation compare(byte[] minPacked, byte[] maxPacked) {
+ int min = bytesToInt(minPacked, 0);
+ int max = bytesToInt(maxPacked, 0);
+ assert max >= min;
+ System.out.println("compare: min=" + min + " max=" + max + " vs queryMin=" + queryMin + " queryMax=" + queryMax);
+
+ if (max < queryMin || min > queryMax) {
+ return BKDReader.Relation.QUERY_OUTSIDE_CELL;
+ } else if (min >= queryMin && max <= queryMax) {
+ return BKDReader.Relation.CELL_INSIDE_QUERY;
+ } else {
+ return BKDReader.Relation.QUERY_CROSSES_CELL;
+ }
+ }
+ });
+
+ for(int docID=0;docID<100;docID++) {
+ boolean expected = docID >= queryMin && docID <= queryMax;
+ boolean actual = hits.get(docID);
+ assertEquals("docID=" + docID, expected, actual);
+ }
+ }
+ }
+ }
+ */
+
+ public void testBasicInts1D() throws Exception {
+ try (Directory dir = newDirectory()) {
+ BKDWriter w = new BKDWriter(1, 4, 2, 1.0f);
+ byte[] scratch = new byte[4];
+ for(int docID=0;docID<100;docID++) {
+ intToBytes(docID, scratch, 0);
+ w.add(scratch, docID);
+ }
+
+ long indexFP;
+ try (IndexOutput out = dir.createOutput("bkd", IOContext.DEFAULT)) {
+ indexFP = w.finish(out);
+ }
+
+ try (IndexInput in = dir.openInput("bkd", IOContext.DEFAULT)) {
+ in.seek(indexFP);
+ BKDReader r = new BKDReader(in, 100);
+
+ // Simple 1D range query:
+ final int queryMin = 42;
+ final int queryMax = 87;
+
+ final BitSet hits = new BitSet();
+ r.intersect(new BKDReader.IntersectVisitor() {
+ @Override
+ public void visit(int docID) {
+ hits.set(docID);
+ System.out.println("visit docID=" + docID);
+ }
+
+ @Override
+ public void visit(int docID, byte[] packedValue) {
+ int x = bytesToInt(packedValue, 0);
+ System.out.println("visit docID=" + docID + " x=" + x);
+ if (x >= queryMin && x <= queryMax) {
+ hits.set(docID);
+ }
+ }
+
+ @Override
+ public BKDReader.Relation compare(byte[] minPacked, byte[] maxPacked) {
+ int min = bytesToInt(minPacked, 0);
+ int max = bytesToInt(maxPacked, 0);
+ assert max >= min;
+ System.out.println("compare: min=" + min + " max=" + max + " vs queryMin=" + queryMin + " queryMax=" + queryMax);
+
+ if (max < queryMin || min > queryMax) {
+ return BKDReader.Relation.QUERY_OUTSIDE_CELL;
+ } else if (min >= queryMin && max <= queryMax) {
+ return BKDReader.Relation.CELL_INSIDE_QUERY;
+ } else {
+ return BKDReader.Relation.QUERY_CROSSES_CELL;
+ }
+ }
+ });
+
+ for(int docID=0;docID<100;docID++) {
+ boolean expected = docID >= queryMin && docID <= queryMax;
+ boolean actual = hits.get(docID);
+ assertEquals("docID=" + docID, expected, actual);
+ }
+ }
+ }
+ }
+
+ public void testRandomIntsNDims() throws Exception {
+ try (Directory dir = newDirectory()) {
+ int numDims = TestUtil.nextInt(random(), 1, 5);
+ int maxPointsInLeafNode = TestUtil.nextInt(random(), 50, 100);
+ float maxMB = (float) 0.1 + (3*random().nextFloat());
+ BKDWriter w = new BKDWriter(numDims, 4, maxPointsInLeafNode, maxMB);
+
+ int numDocs = atLeast(1000);
+ if (VERBOSE) {
+ System.out.println("TEST: numDims=" + numDims + " numDocs=" + numDocs);
+ }
+ int[][] docs = new int[numDocs][];
+ byte[] scratch = new byte[4*numDims];
+ for(int docID=0;docID<numDocs;docID++) {
+ int[] values = new int[numDims];
+ for(int dim=0;dim<numDims;dim++) {
+ values[dim] = random().nextInt();
+ intToBytes(values[dim], scratch, dim);
+ }
+ docs[docID] = values;
+ w.add(scratch, docID);
+ }
+
+ long indexFP;
+ try (IndexOutput out = dir.createOutput("bkd", IOContext.DEFAULT)) {
+ indexFP = w.finish(out);
+ }
+
+ try (IndexInput in = dir.openInput("bkd", IOContext.DEFAULT)) {
+ in.seek(indexFP);
+ BKDReader r = new BKDReader(in, 100);
+
+ int iters = atLeast(100);
+ for(int iter=0;iter<iters;iter++) {
+ if (VERBOSE) {
+ System.out.println("\nTEST: iter=" + iter);
+ }
+
+ // Random N dime rect query:
+ int[] queryMin = new int[numDims];
+ int[] queryMax = new int[numDims];
+ for(int dim=0;dim<numDims;dim++) {
+ queryMin[dim] = random().nextInt();
+ queryMax[dim] = random().nextInt();
+ if (queryMin[dim] > queryMax[dim]) {
+ int x = queryMin[dim];
+ queryMin[dim] = queryMax[dim];
+ queryMax[dim] = x;
+ }
+ }
+
+ final BitSet hits = new BitSet();
+ r.intersect(new BKDReader.IntersectVisitor() {
+ @Override
+ public void visit(int docID) {
+ hits.set(docID);
+ //System.out.println("visit docID=" + docID);
+ }
+
+ @Override
+ public void visit(int docID, byte[] packedValue) {
+ //System.out.println("visit check docID=" + docID);
+ for(int dim=0;dim<numDims;dim++) {
+ int x = bytesToInt(packedValue, dim);
+ if (x < queryMin[dim] || x > queryMax[dim]) {
+ //System.out.println(" no");
+ return;
+ }
+ }
+
+ //System.out.println(" yes");
+ hits.set(docID);
+ }
+
+ @Override
+ public BKDReader.Relation compare(byte[] minPacked, byte[] maxPacked) {
+ boolean crosses = false;
+ for(int dim=0;dim<numDims;dim++) {
+ int min = bytesToInt(minPacked, dim);
+ int max = bytesToInt(maxPacked, dim);
+ assert max >= min;
+
+ if (max < queryMin[dim] || min > queryMax[dim]) {
+ return BKDReader.Relation.QUERY_OUTSIDE_CELL;
+ } else if (min < queryMin[dim] || max > queryMax[dim]) {
+ crosses = true;
+ }
+ }
+
+ if (crosses) {
+ return BKDReader.Relation.QUERY_CROSSES_CELL;
+ } else {
+ return BKDReader.Relation.CELL_INSIDE_QUERY;
+ }
+ }
+ });
+
+ for(int docID=0;docID<100;docID++) {
+ int[] docValues = docs[docID];
+ boolean expected = true;
+ for(int dim=0;dim<numDims;dim++) {
+ int x = docValues[dim];
+ if (x < queryMin[dim] || x > queryMax[dim]) {
+ expected = false;
+ break;
+ }
+ }
+ boolean actual = hits.get(docID);
+ assertEquals("docID=" + docID, expected, actual);
+ }
+ }
+ }
+ }
+ }
+}