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/19 01:15:44 UTC
svn commit: r1709330 - in /lucene/dev/branches/lucene6825/lucene:
core/src/java/org/apache/lucene/util/
core/src/java/org/apache/lucene/util/bkd/
core/src/test/org/apache/lucene/util/bkd/
suggest/src/java/org/apache/lucene/search/suggest/fst/ test-fram...
Author: mikemccand
Date: Sun Oct 18 23:15:44 2015
New Revision: 1709330
URL: http://svn.apache.org/viewvc?rev=1709330&view=rev
Log:
LUCENE-6825: remove all nocommits; add missing MDW.createTempOutput wrapping; fix double-write per dim during build
Modified:
lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java
lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/BKDReader.java
lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/OfflinePointReader.java
lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/OfflinePointWriter.java
lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/PointReader.java
lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/PointWriter.java
lucene/dev/branches/lucene6825/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKD.java
lucene/dev/branches/lucene6825/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java
lucene/dev/branches/lucene6825/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
Modified: lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java?rev=1709330&r1=1709329&r2=1709330&view=diff
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java (original)
+++ lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java Sun Oct 18 23:15:44 2015
@@ -186,6 +186,7 @@ public final class ByteBlockPool {
}
}
}
+
/**
* Advances the pool to its next buffer. This method should be called once
* after the constructor to initialize the pool. In contrast to the
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=1709330&r1=1709329&r2=1709330&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 Sun Oct 18 23:15:44 2015
@@ -43,7 +43,7 @@ import org.apache.lucene.store.TrackingD
* @lucene.experimental
* @lucene.internal
*/
-public final class OfflineSorter {
+public class OfflineSorter {
/** Convenience constant for megabytes */
public final static long MB = 1024 * 1024;
@@ -237,7 +237,7 @@ public final class OfflineSorter {
TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(dir);
boolean success = false;
- try (ByteSequencesReader is = new ByteSequencesReader(dir.openInput(inputFileName, IOContext.READONCE))) {
+ try (ByteSequencesReader is = getReader(dir.openInput(inputFileName, IOContext.READONCE))) {
int lineCount;
while ((lineCount = readPartition(is)) > 0) {
@@ -284,8 +284,9 @@ public final class OfflineSorter {
protected String sortPartition(TrackingDirectoryWrapper trackingDir) throws IOException {
BytesRefArray data = this.buffer;
- try (IndexOutput tempFile = trackingDir.createTempOutput(tempFileNamePrefix, "sort", IOContext.DEFAULT)) {
- ByteSequencesWriter out = new ByteSequencesWriter(tempFile);
+ try (IndexOutput tempFile = trackingDir.createTempOutput(tempFileNamePrefix, "sort", IOContext.DEFAULT);
+ ByteSequencesWriter out = getWriter(tempFile);) {
+
BytesRef spare;
long start = System.currentTimeMillis();
@@ -320,16 +321,18 @@ public final class OfflineSorter {
String newSegmentName = null;
- try (IndexOutput out = trackingDir.createTempOutput(tempFileNamePrefix, "sort", IOContext.DEFAULT)) {
- newSegmentName = out.getName();
- ByteSequencesWriter writer = new ByteSequencesWriter(out);
+ try (IndexOutput out = trackingDir.createTempOutput(tempFileNamePrefix, "sort", IOContext.DEFAULT);
+ ByteSequencesWriter writer = getWriter(out);) {
+ newSegmentName = out.getName();
+
// Open streams and read the top for each file
for (int i = 0; i < segments.size(); i++) {
- streams[i] = new ByteSequencesReader(dir.openInput(segments.get(i), IOContext.READONCE));
- byte[] line = streams[i].read();
- assert line != null;
- queue.insertWithOverflow(new FileAndTop(i, line));
+ streams[i] = getReader(dir.openInput(segments.get(i), IOContext.READONCE));
+ BytesRefBuilder bytes = new BytesRefBuilder();
+ boolean result = streams[i].read(bytes);
+ assert result;
+ queue.insertWithOverflow(new FileAndTop(i, bytes));
}
// Unix utility sort() uses ordered array of files to pick the next line from, updating
@@ -363,13 +366,12 @@ public final class OfflineSorter {
/** Read in a single partition of data */
int readPartition(ByteSequencesReader reader) throws IOException {
long start = System.currentTimeMillis();
- final BytesRef scratch = new BytesRef();
- while ((scratch.bytes = reader.read()) != null) {
- scratch.length = scratch.bytes.length;
- buffer.append(scratch);
+ final BytesRefBuilder scratch = new BytesRefBuilder();
+ while (reader.read(scratch)) {
+ buffer.append(scratch.get());
// Account for the created objects.
// (buffer slots do not account to buffer size.)
- if (ramBufferSize.bytes < bufferBytesUsed.get()) {
+ if (bufferBytesUsed.get() > ramBufferSize.bytes) {
break;
}
}
@@ -381,19 +383,28 @@ public final class OfflineSorter {
final int fd;
final BytesRefBuilder current;
- FileAndTop(int fd, byte[] firstLine) {
+ FileAndTop(int fd, BytesRefBuilder firstLine) {
this.fd = fd;
- this.current = new BytesRefBuilder();
- this.current.copyBytes(firstLine, 0, firstLine.length);
+ this.current = firstLine;
}
}
+ /** Subclasses can override to change how byte sequences are written to disk. */
+ protected ByteSequencesWriter getWriter(IndexOutput out) throws IOException {
+ return new ByteSequencesWriter(out);
+ }
+
+ /** Subclasses can override to change how byte sequences are read from disk. */
+ protected ByteSequencesReader getReader(IndexInput in) throws IOException {
+ return new ByteSequencesReader(in);
+ }
+
/**
* Utility class to emit length-prefixed byte[] entries to an output stream for sorting.
* Complementary to {@link ByteSequencesReader}.
*/
public static class ByteSequencesWriter implements Closeable {
- private final IndexOutput out;
+ protected final IndexOutput out;
/** Constructs a ByteSequencesWriter to the provided DataOutput */
public ByteSequencesWriter(IndexOutput out) {
@@ -448,7 +459,7 @@ public final class OfflineSorter {
* Complementary to {@link ByteSequencesWriter}.
*/
public static class ByteSequencesReader implements Closeable {
- private final IndexInput in;
+ protected final IndexInput in;
/** Constructs a ByteSequencesReader from the provided IndexInput */
public ByteSequencesReader(IndexInput in) {
@@ -478,29 +489,6 @@ public final class OfflineSorter {
}
/**
- * Reads the next entry and returns it if successful.
- *
- * @see #read(BytesRefBuilder)
- *
- * @return Returns <code>null</code> if EOF occurred before the next entry
- * could be read.
- * @throws EOFException if the file ends before the full sequence is read.
- */
- public byte[] read() throws IOException {
- short length;
- try {
- length = in.readShort();
- } catch (EOFException e) {
- return null;
- }
-
- assert length >= 0 : "Sanity: sequence length < 0: " + length;
- byte[] result = new byte[length];
- in.readBytes(result, 0, length);
- return result;
- }
-
- /**
* Closes the provided {@link IndexInput}.
*/
@Override
Modified: 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=1709330&r1=1709329&r2=1709330&view=diff
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/BKDReader.java (original)
+++ lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/BKDReader.java Sun Oct 18 23:15:44 2015
@@ -25,10 +25,6 @@ import org.apache.lucene.store.IndexInpu
import org.apache.lucene.util.Accountable;
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 */
@@ -196,7 +192,8 @@ public final class BKDReader implements
// How many points are stored in this leaf cell:
int count = state.in.readVInt();
- // nocommit can we get this back?
+ // TODO: we could maybe pollute the IntersectVisitor API with a "grow" method if this maybe helps perf
+ // enough (it did before, esp. for the 1D case):
//state.docs.grow(count);
int docID = 0;
for(int i=0;i<count;i++) {
@@ -218,10 +215,11 @@ public final class BKDReader implements
int splitDim = splitPackedValues[address] & 0xff;
assert splitDim < numDims;
- // nocommit can we alloc & reuse this up front?
+ // TODO: can we alloc & reuse this up front?
byte[] splitValue = new byte[bytesPerDim];
System.arraycopy(splitPackedValues, address+1, splitValue, 0, bytesPerDim);
+ // TODO: can we alloc & reuse this up front?
byte[] splitPackedValue = new byte[packedBytesLength];
if (BKDUtil.compare(bytesPerDim, state.minPacked, splitDim, splitValue, 0) <= 0) {
Modified: 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=1709330&r1=1709329&r2=1709330&view=diff
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java (original)
+++ lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java Sun Oct 18 23:15:44 2015
@@ -24,14 +24,13 @@ import java.util.Comparator;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.store.ByteArrayDataInput;
-import org.apache.lucene.store.ByteArrayDataOutput;
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.store.TrackingDirectoryWrapper;
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;
@@ -41,7 +40,7 @@ import org.apache.lucene.util.RamUsageEs
// 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
+// - allow variable length byte[] (across docs and dims), 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
@@ -85,9 +84,6 @@ public final class BKDWriter implements
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;
@@ -100,13 +96,12 @@ public final class BKDWriter implements
final TrackingDirectoryWrapper tempDir;
final String tempFileNamePrefix;
- final byte[] scratchMax;
final byte[] scratchDiff;
final byte[] scratchPackedValue;
final byte[] scratch1;
final byte[] scratch2;
- private OfflineSorter.ByteSequencesWriter offlinePointWriter;
+ private OfflinePointWriter offlinePointWriter;
private HeapPointWriter heapPointWriter;
private IndexOutput tempInput;
@@ -119,7 +114,7 @@ public final class BKDWriter implements
this(tempDir, tempFileNamePrefix, numDims, bytesPerDim, DEFAULT_MAX_POINTS_IN_LEAF_NODE, DEFAULT_MAX_MB_SORT_IN_HEAP);
}
- public BKDWriter(Directory tempDir, String tempFileNamePrefix, int numDims, int bytesPerDim, int maxPointsInLeafNode, float maxMBSortInHeap) throws IOException {
+ public BKDWriter(Directory tempDir, String tempFileNamePrefix, int numDims, int bytesPerDim, int maxPointsInLeafNode, double maxMBSortInHeap) throws IOException {
verifyParams(numDims, maxPointsInLeafNode, maxMBSortInHeap);
// We use tracking dir to deal with removing files on exception, so each place that
// creates temp files doesn't need crazy try/finally/sucess logic:
@@ -130,7 +125,6 @@ public final class BKDWriter implements
this.bytesPerDim = bytesPerDim;
packedBytesLength = numDims * bytesPerDim;
- scratchMax = new byte[bytesPerDim];
scratchDiff = new byte[bytesPerDim];
scratchPackedValue = new byte[packedBytesLength];
scratch1 = new byte[packedBytesLength];
@@ -138,18 +132,28 @@ public final class BKDWriter implements
// 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));
+ // As we recurse, we compute temporary partitions of the data, halving the
+ // number of points at each recursion. Once there are few enough points,
+ // we can switch to sorting in heap instead of offline (on disk). At any
+ // time in the recursion, we hold the number of points at that level, plus
+ // all recursive halves (i.e. 16 + 8 + 4 + 2) so the memory usage is 2X
+ // what that level would consume, so we multiply by 0.5 to convert from
+ // bytes to points here. Each dimension has its own sorted partition, so
+ // we must divide by numDims as wel.
+
+ maxPointsSortInHeap = (int) (0.5 * (maxMBSortInHeap * 1024 * 1024) / (bytesPerDoc * numDims));
+
+ // Finally, we must be able to hold at least the leaf node in heap during build:
+ if (maxPointsSortInHeap < maxPointsInLeafNode) {
+ throw new IllegalArgumentException("maxMBSortInHeap=" + maxMBSortInHeap + " only allows for maxPointsSortInHeap=" + maxPointsSortInHeap + ", but this is less than maxPointsInLeafNode=" + maxPointsInLeafNode + "; either increase maxMBSortInHeap or decrease maxPointsInLeafNode");
+ }
// We write first maxPointsSortInHeap in heap, then cutover to offline for additional points:
heapPointWriter = new HeapPointWriter(16, maxPointsSortInHeap, packedBytesLength);
}
- public static void verifyParams(int numDims, int maxPointsInLeafNode, float maxMBSortInHeap) {
+ public static void verifyParams(int numDims, int maxPointsInLeafNode, double 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) {
@@ -171,20 +175,12 @@ public final class BKDWriter implements
// For each .add we just append to this input file, then in .finish we sort this input and resursively build the tree:
tempInput = tempDir.createTempOutput(tempFileNamePrefix, "bkd", IOContext.DEFAULT);
- offlinePointWriter = new OfflineSorter.ByteSequencesWriter(tempInput);
+ offlinePointWriter = new OfflinePointWriter(tempDir, tempInput, packedBytesLength);
PointReader reader = heapPointWriter.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(heapPointWriter.docIDs[i]);
- scratchBytesOutput.writeVLong(i);
- // TODO: can/should OfflineSorter optimize the fixed-width case?
- offlinePointWriter.write(scratchBytes, 0, scratchBytes.length);
+ offlinePointWriter.append(reader.packedValue(), i, heapPointWriter.docIDs[i]);
}
heapPointWriter = null;
@@ -199,11 +195,7 @@ public final class BKDWriter implements
if (offlinePointWriter == null) {
switchToOffline();
}
- scratchBytesOutput.reset(scratchBytes);
- scratchBytesOutput.writeBytes(packedValue, 0, packedValue.length);
- scratchBytesOutput.writeVInt(docID);
- scratchBytesOutput.writeVLong(pointCount);
- offlinePointWriter.write(scratchBytes, 0, scratchBytes.length);
+ offlinePointWriter.append(packedValue, pointCount, docID);
} else {
// Not too many points added yet, continue using heap:
heapPointWriter.append(packedValue, pointCount, docID);
@@ -213,33 +205,7 @@ public final class BKDWriter implements
}
// 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 PointWriter convertToFixedWidth(String in) throws IOException {
- BytesRefBuilder scratch = new BytesRefBuilder();
- scratch.grow(bytesPerDoc);
- BytesRef bytes = scratch.get();
- ByteArrayDataInput dataReader = new ByteArrayDataInput();
-
- byte[] packedValue = new byte[packedBytesLength];
- try (
- OfflineSorter.ByteSequencesReader reader = new OfflineSorter.ByteSequencesReader(tempDir.openInput(in, IOContext.READONCE));
- PointWriter sortedPointWriter = getPointWriter(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;
- sortedPointWriter.append(packedValue, ord, docID);
- }
- return sortedPointWriter;
- }
- }
+ // encoding and not have our own ByteSequencesReader/Writer
/** If dim=-1 we sort by docID, else by that dim. */
private void sortHeapPointWriter(final HeapPointWriter writer, int start, int length, int dim) {
@@ -294,6 +260,9 @@ public final class BKDWriter implements
if (heapPointWriter != null) {
+ assert tempInput == null;
+
+ // We never spilled the incoming points to disk, so now we sort in heap:
HeapPointWriter sorted;
if (dim == 0) {
@@ -346,14 +315,24 @@ public final class BKDWriter implements
}
};
- OfflineSorter sorter = new OfflineSorter(tempDir, tempFileNamePrefix, cmp);
- String sortedFileName = sorter.sort(tempInput.getName());
+ // TODO: this is sort of sneaky way to get the final OfflinePointWriter from OfflineSorter:
+ OfflinePointWriter[] lastWriter = new OfflinePointWriter[1];
- try (PointWriter writer = convertToFixedWidth(sortedFileName)) {
- // nocommit can we avoid the double-write here? i.e. have offline sorter write fixed width to begin with?
- tempDir.deleteFile(sortedFileName);
- return writer;
- }
+ OfflineSorter sorter = new OfflineSorter(tempDir, tempFileNamePrefix, cmp) {
+ @Override
+ protected ByteSequencesWriter getWriter(IndexOutput out) {
+ lastWriter[0] = new OfflinePointWriter(tempDir, out, packedBytesLength);
+ return lastWriter[0];
+ }
+ @Override
+ protected ByteSequencesReader getReader(IndexInput in) throws IOException {
+ return new OfflinePointReader(in, packedBytesLength, 0, -1);
+ }
+ };
+ sorter.sort(tempInput.getName());
+ assert lastWriter[0] != null;
+
+ return lastWriter[0];
}
}
@@ -469,6 +448,7 @@ public final class BKDWriter implements
tempInput.close();
} finally {
tempDir.deleteFile(tempInput.getName());
+ tempInput = null;
}
}
}
@@ -494,17 +474,15 @@ public final class BKDWriter implements
/** 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:
+ // Read the split value, then mark all ords in the right tree (larger than the split value):
try (PointReader reader = source.writer.getReader(source.start + source.count - rightCount)) {
boolean result = reader.next();
assert result;
- System.arraycopy(reader.packedValue(), splitDim*bytesPerDim, scratchMax, 0, bytesPerDim);
+ System.arraycopy(reader.packedValue(), splitDim*bytesPerDim, scratch1, 0, bytesPerDim);
ordBitSet.set(reader.ord());
@@ -518,8 +496,7 @@ public final class BKDWriter implements
assert rightCount == ordBitSet.cardinality(): "rightCount=" + rightCount + " cardinality=" + ordBitSet.cardinality();
- // nocommit rename scratchMax
- return scratchMax;
+ return scratch1;
}
/** Called only in assert */
@@ -542,8 +519,8 @@ public final class BKDWriter implements
int splitDim = -1;
for(int dim=0;dim<numDims;dim++) {
BKDUtil.subtract(bytesPerDim, dim, maxPackedValue, minPackedValue, scratchDiff);
- if (splitDim == -1 || BKDUtil.compare(bytesPerDim, scratchDiff, 0, scratchMax, 0) > 0) {
- System.arraycopy(scratchDiff, 0, scratchMax, 0, bytesPerDim);
+ if (splitDim == -1 || BKDUtil.compare(bytesPerDim, scratchDiff, 0, scratch1, 0) > 0) {
+ System.arraycopy(scratchDiff, 0, scratch1, 0, bytesPerDim);
splitDim = dim;
}
}
@@ -551,6 +528,23 @@ public final class BKDWriter implements
return splitDim;
}
+ /** Only called in the 1D case, to pull a partition back into heap once
+ * the point count is low enough while recursing. */
+ private PathSlice switchToHeap(PathSlice source) throws IOException {
+ int count = Math.toIntExact(source.count);
+ try (
+ PointWriter writer = new HeapPointWriter(count, count, packedBytesLength);
+ PointReader reader = source.writer.getReader(source.start);
+ ) {
+ for(int i=0;i<count;i++) {
+ boolean hasNext = reader.next();
+ assert hasNext;
+ writer.append(reader.packedValue(), reader.ord(), reader.docID());
+ }
+ return new PathSlice(writer, 0, count);
+ }
+ }
+
/** The array (sized numDims) of PathSlice describe the cell we have currently recursed to. */
private void build(int nodeID, int leafNodeOffset,
PathSlice[] slices,
@@ -571,14 +565,23 @@ public final class BKDWriter implements
}
}
+ if (numDims == 1 && slices[0].writer instanceof OfflinePointWriter && slices[0].count <= maxPointsSortInHeap) {
+ // Special case for 1D, to cutover to heap once we recurse deeply enough:
+ slices[0] = switchToHeap(slices[0]);
+ }
+
if (nodeID >= leafNodeOffset) {
// Leaf node: write block
if (DEBUG) System.out.println(" leaf");
PathSlice source = slices[0];
+ if (source.writer instanceof HeapPointWriter == false) {
+ // Adversarial cases can cause this, e.g. very lopsided data, all equal points
+ source = switchToHeap(source);
+ }
+
// We ensured that maxPointsSortInHeap was >= maxPointsInLeafNode, so we better be in heap at this point:
- assert source.writer instanceof HeapPointWriter;
HeapPointWriter heapSource = (HeapPointWriter) source.writer;
// Sort by docID in the leaf so we can delta-vInt encode:
@@ -619,8 +622,6 @@ public final class BKDWriter implements
PathSlice source = slices[splitDim];
- //System.out.println(" splitDim=" + splitDim + " source=" + source + " right docs:");
-
assert nodeID < splitPackedValues.length: "nodeID=" + nodeID + " splitValues.length=" + splitPackedValues.length;
// How many points will be in the left tree:
@@ -628,7 +629,6 @@ public final class BKDWriter implements
long leftCount = source.count - rightCount;
byte[] splitValue = markRightTree(rightCount, splitDim, source, ordBitSet);
- //System.out.println(" split value=" + BKDUtil.bytesToInt(splitValue, 0));
int address = nodeID * (1+bytesPerDim);
splitPackedValues[address] = (byte) splitDim;
System.arraycopy(splitValue, 0, splitPackedValues, address + 1, bytesPerDim);
@@ -655,8 +655,6 @@ public final class BKDWriter implements
continue;
}
- //System.out.println(" partition dim=" + dim);
-
try (PointWriter leftPointWriter = getPointWriter(leftCount);
PointWriter rightPointWriter = getPointWriter(source.count - leftCount);
PointReader reader = slices[dim].writer.getReader(slices[dim].start);) {
@@ -686,7 +684,6 @@ public final class BKDWriter implements
}
}
- //System.out.println("pointCount=" + pointCount);
ordBitSet.clear(0, pointCount);
// Recurse on left tree:
@@ -695,17 +692,20 @@ public final class BKDWriter implements
minPackedValue, maxSplitPackedValue,
splitPackedValues, leafBlockFPs);
for(int dim=0;dim<numDims;dim++) {
+ // Don't destroy the dim we split on because we just re-used what our caller above gave us for that dim:
if (dim != splitDim) {
leftSlices[dim].writer.destroy();
}
}
+ // TODO: we could "tail recurse" here? have our parent discard its refs as we recurse right?
// Recurse on right tree:
build(2*nodeID+1, leafNodeOffset, rightSlices,
ordBitSet, out,
minSplitPackedValue, maxPackedValue,
splitPackedValues, leafBlockFPs);
for(int dim=0;dim<numDims;dim++) {
+ // Don't destroy the dim we split on because we just re-used what our caller above gave us for that dim:
if (dim != splitDim) {
rightSlices[dim].writer.destroy();
}
@@ -718,7 +718,7 @@ public final class BKDWriter implements
int size = Math.toIntExact(count);
return new HeapPointWriter(size, size, packedBytesLength);
} else {
- return new OfflinePointWriter(tempDir, tempFileNamePrefix, count, packedBytesLength);
+ return new OfflinePointWriter(tempDir, tempFileNamePrefix, packedBytesLength);
}
}
}
Modified: lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/OfflinePointReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/OfflinePointReader.java?rev=1709330&r1=1709329&r2=1709330&view=diff
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/OfflinePointReader.java (original)
+++ lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/OfflinePointReader.java Sun Oct 18 23:15:44 2015
@@ -17,20 +17,17 @@ package org.apache.lucene.util.bkd;
* limitations under the License.
*/
-import java.io.BufferedInputStream;
+import java.io.EOFException;
import java.io.IOException;
-import java.io.InputStream;
-import java.nio.file.Files;
-import java.nio.file.Path;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.OfflineSorter;
import org.apache.lucene.util.RamUsageEstimator;
-final class OfflinePointReader implements PointReader {
- final IndexInput in;
+final class OfflinePointReader extends OfflineSorter.ByteSequencesReader implements PointReader {
long countLeft;
private final byte[] packedValue;
private long ord;
@@ -38,8 +35,12 @@ final class OfflinePointReader implement
final int bytesPerDoc;
OfflinePointReader(Directory tempDir, String tempFileName, int packedBytesLength, long start, long length) throws IOException {
+ this(tempDir.openInput(tempFileName, IOContext.READONCE), packedBytesLength, start, length);
+ }
+
+ OfflinePointReader(IndexInput in, int packedBytesLength, long start, long length) throws IOException {
+ super(in);
bytesPerDoc = packedBytesLength + RamUsageEstimator.NUM_BYTES_LONG + RamUsageEstimator.NUM_BYTES_INT;
- in = tempDir.openInput(tempFileName, IOContext.READONCE);
long seekFP = start * bytesPerDoc;
in.seek(seekFP);
this.countLeft = length;
@@ -48,16 +49,35 @@ final class OfflinePointReader implement
@Override
public boolean next() throws IOException {
- if (countLeft == 0) {
+ if (countLeft >= 0) {
+ if (countLeft == 0) {
+ return false;
+ }
+ countLeft--;
+ }
+ try {
+ in.readBytes(packedValue, 0, packedValue.length);
+ } catch (EOFException eofe) {
+ assert countLeft == -1;
return false;
}
- countLeft--;
- in.readBytes(packedValue, 0, packedValue.length);
ord = in.readLong();
docID = in.readInt();
return true;
}
+ @Override
+ public boolean read(BytesRefBuilder ref) throws IOException {
+ ref.grow(bytesPerDoc);
+ try {
+ in.readBytes(ref.bytes(), 0, bytesPerDoc);
+ } catch (EOFException eofe) {
+ return false;
+ }
+ ref.setLength(bytesPerDoc);
+ return true;
+ }
+
@Override
public byte[] packedValue() {
return packedValue;
Modified: lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/OfflinePointWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/OfflinePointWriter.java?rev=1709330&r1=1709329&r2=1709330&view=diff
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/OfflinePointWriter.java (original)
+++ lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/OfflinePointWriter.java Sun Oct 18 23:15:44 2015
@@ -17,36 +17,32 @@ package org.apache.lucene.util.bkd;
* 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.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
-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 OfflinePointWriter implements PointWriter {
+final class OfflinePointWriter extends OfflineSorter.ByteSequencesWriter implements PointWriter {
final Directory tempDir;
- final IndexOutput out;
- final long count;
final int packedBytesLength;
final int bytesPerDoc;
- private long countWritten;
+ private long count;
private boolean closed;
- public OfflinePointWriter(Directory tempDir, String tempFileNamePrefix, long count, int packedBytesLength) throws IOException {
+ public OfflinePointWriter(Directory tempDir, String tempFileNamePrefix, int packedBytesLength) throws IOException {
+ this(tempDir, tempDir.createTempOutput(tempFileNamePrefix, "bkd", IOContext.DEFAULT), packedBytesLength);
+ }
+
+ public OfflinePointWriter(Directory tempDir, IndexOutput out, int packedBytesLength) {
+ super(out);
this.tempDir = tempDir;
- out = tempDir.createTempOutput(tempFileNamePrefix, "bkd", IOContext.DEFAULT);
this.packedBytesLength = packedBytesLength;
bytesPerDoc = packedBytesLength + RamUsageEstimator.NUM_BYTES_LONG + RamUsageEstimator.NUM_BYTES_INT;
- this.count = count;
}
@Override
@@ -55,7 +51,16 @@ final class OfflinePointWriter implement
out.writeBytes(packedValue, 0, packedValue.length);
out.writeLong(ord);
out.writeInt(docID);
- countWritten++;
+ count++;
+ }
+
+ @Override
+ public void write(byte[] bytes, int off, int len) throws IOException {
+ if (len != bytesPerDoc) {
+ throw new IllegalArgumentException("len=" + len + " bytesPerDoc=" + bytesPerDoc);
+ }
+ out.writeBytes(bytes, off, len);
+ count++;
}
@Override
@@ -66,11 +71,8 @@ final class OfflinePointWriter implement
@Override
public void close() throws IOException {
+ super.close();
closed = true;
- out.close();
- if (count != countWritten) {
- throw new IllegalStateException("wrote " + countWritten + " values, but expected " + count);
- }
}
@Override
Modified: lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/PointReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/PointReader.java?rev=1709330&r1=1709329&r2=1709330&view=diff
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/PointReader.java (original)
+++ lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/PointReader.java Sun Oct 18 23:15:44 2015
@@ -20,13 +20,21 @@ package org.apache.lucene.util.bkd;
import java.io.Closeable;
import java.io.IOException;
-/** Iterates through all points written with a {@link PointWriter}, and abstracts away
- * whether points a read from disk or simple arrays in heap. */
+/** One pass iterator through all points previously written with a
+ * {@link PointWriter}, abstracting away whether points a read
+ * from (offline) disk or simple arrays in heap. */
interface PointReader extends Closeable {
+
+ /** Returns false once iteration is done, else true. */
boolean next() throws IOException;
- // nocommit maybe caller should pass in reused buffer? reduces GC on recursion...?
+
+ /** Returns the packed byte[] value */
byte[] packedValue();
+
+ /** Point ordinal */
long ord();
+
+ /** DocID for this point */
int docID();
}
Modified: lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/PointWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/PointWriter.java?rev=1709330&r1=1709329&r2=1709330&view=diff
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/PointWriter.java (original)
+++ lucene/dev/branches/lucene6825/lucene/core/src/java/org/apache/lucene/util/bkd/PointWriter.java Sun Oct 18 23:15:44 2015
@@ -25,7 +25,7 @@ import java.io.IOException;
* in heap. */
interface PointWriter extends Closeable {
void append(byte[] packedValue, long ord, int docID) throws IOException;
- PointReader getReader(long startDoc) throws IOException;
+ PointReader getReader(long startPoint) throws IOException;
void destroy() throws IOException;
}
Modified: 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=1709330&r1=1709329&r2=1709330&view=diff
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKD.java (original)
+++ lucene/dev/branches/lucene6825/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKD.java Sun Oct 18 23:15:44 2015
@@ -17,8 +17,10 @@ package org.apache.lucene.util.bkd;
* limitations under the License.
*/
+import java.io.IOException;
import java.math.BigInteger;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.BitSet;
import java.util.List;
@@ -28,6 +30,7 @@ import org.apache.lucene.store.IndexInpu
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
@@ -46,8 +49,8 @@ public class TestBKD extends LuceneTestC
long indexFP;
try (IndexOutput out = dir.createOutput("bkd", IOContext.DEFAULT)) {
- indexFP = w.finish(out);
- }
+ indexFP = w.finish(out);
+ }
try (IndexInput in = dir.openInput("bkd", IOContext.DEFAULT)) {
in.seek(indexFP);
@@ -348,15 +351,60 @@ public class TestBKD extends LuceneTestC
}
}
- // nocommit testAccountableHasDelegate?
+ /** Make sure we close open files, delete temp files, etc., on exception */
+ public void testWithExceptions() throws Exception {
+ int numDocs = atLeast(10000);
+ int numBytesPerDim = TestUtil.nextInt(random(), 2, 30);
+ int numDims = TestUtil.nextInt(random(), 1, 5);
+
+ byte[][][] docValues = new byte[numDocs][][];
+
+ for(int docID=0;docID<numDocs;docID++) {
+ byte[][] values = new byte[numDims][];
+ for(int dim=0;dim<numDims;dim++) {
+ values[dim] = new byte[numBytesPerDim];
+ random().nextBytes(values[dim]);
+ }
+ docValues[docID] = values;
+ }
- // nocommit test on exception
+ double maxMBHeap = 0.05;
+ // Keep retrying until we 1) we allow a big enough heap, and 2) we hit a random IOExc from MDW:
+ boolean done = false;
+ while (done == false) {
+ try (MockDirectoryWrapper dir = newMockFSDirectory(createTempDir())) {
+ try {
+ dir.setRandomIOExceptionRate(0.05);
+ dir.setRandomIOExceptionRateOnOpen(0.05);
+ if (dir instanceof MockDirectoryWrapper) {
+ dir.setEnableVirusScanner(false);
+ }
+ verify(dir, docValues, null, numDims, numBytesPerDim, 50, maxMBHeap);
+ } catch (IllegalArgumentException iae) {
+ // This just means we got a too-small maxMB for the maxPointsInLeafNode; just retry w/ more heap
+ assertTrue(iae.getMessage().contains("either increase maxMBSortInHeap or decrease maxPointsInLeafNode"));
+ System.out.println(" more heap");
+ maxMBHeap *= 1.25;
+ } catch (IOException ioe) {
+ if (ioe.getMessage().contains("a random IOException")) {
+ // BKDWriter should fully clean up after itself:
+ done = true;
+ } else {
+ throw ioe;
+ }
+ }
+
+ String[] files = dir.listAll();
+ assertTrue("files=" + Arrays.toString(files), files.length == 0 || Arrays.equals(files, new String[] {"extra0"}));
+ }
+ }
+ }
public void testRandomBinaryTiny() throws Exception {
doTestRandomBinary(10);
}
- public void testRandomBinarydMediumTiny() throws Exception {
+ public void testRandomBinarydMedium() throws Exception {
doTestRandomBinary(10000);
}
@@ -365,6 +413,16 @@ public class TestBKD extends LuceneTestC
doTestRandomBinary(200000);
}
+ public void testTooLittleHeap() throws Exception {
+ try (Directory dir = getDirectory(0)) {
+ new BKDWriter(dir, "bkd", 1, 16, 1000000, 0.001);
+ fail("did not hit exception");
+ } catch (IllegalArgumentException iae) {
+ // expected
+ assertTrue(iae.getMessage().contains("either increase maxMBSortInHeap or decrease maxPointsInLeafNode"));
+ }
+ }
+
private void doTestRandomBinary(int count) throws Exception {
int numDocs = atLeast(count);
int numBytesPerDim = TestUtil.nextInt(random(), 2, 30);
@@ -462,12 +520,28 @@ public class TestBKD extends LuceneTestC
/** docIDs can be null, for the single valued case, else it maps value to docID */
private void verify(byte[][][] docValues, int[] docIDs, int numDims, int numBytesPerDim) throws Exception {
- int numValues = docValues.length;
-
try (Directory dir = getDirectory(docValues.length)) {
- int maxPointsInLeafNode = TestUtil.nextInt(random(), 50, 100);
- float maxMB = (float) 0.1 + (3*random().nextFloat());
- BKDWriter w = new BKDWriter(dir, "tmp", numDims, numBytesPerDim, maxPointsInLeafNode, maxMB);
+ while (true) {
+ int maxPointsInLeafNode = TestUtil.nextInt(random(), 50, 100);
+ double maxMB = (float) 0.1 + (3*random().nextDouble());
+ try {
+ verify(dir, docValues, docIDs, numDims, numBytesPerDim, maxPointsInLeafNode, maxMB);
+ return;
+ } catch (IllegalArgumentException iae) {
+ // This just means we got a too-small maxMB for the maxPointsInLeafNode; just retry
+ assertTrue(iae.getMessage().contains("either increase maxMBSortInHeap or decrease maxPointsInLeafNode"));
+ }
+ }
+ }
+ }
+
+ private void verify(Directory dir, byte[][][] docValues, int[] docIDs, int numDims, int numBytesPerDim, int maxPointsInLeafNode, double maxMB) throws Exception {
+ int numValues = docValues.length;
+ if (VERBOSE) {
+ System.out.println("TEST: numValues=" + numValues + " numDims=" + numDims + " numBytesPerDim=" + numBytesPerDim + " maxPointsInLeafNode=" + maxPointsInLeafNode + " maxMB=" + maxMB);
+ }
+ long indexFP;
+ try (BKDWriter w = new BKDWriter(dir, "tmp", numDims, numBytesPerDim, maxPointsInLeafNode, maxMB)) {
byte[] scratch = new byte[numBytesPerDim*numDims];
for(int ord=0;ord<numValues;ord++) {
@@ -489,38 +563,44 @@ public class TestBKD extends LuceneTestC
w.add(scratch, docID);
}
- long indexFP;
+ boolean success = false;
try (IndexOutput out = dir.createOutput("bkd", IOContext.DEFAULT)) {
indexFP = w.finish(out);
+ success = true;
+ } finally {
+ if (success == false) {
+ IOUtils.deleteFilesIgnoringExceptions(dir, "bkd");
+ }
}
+ }
- try (IndexInput in = dir.openInput("bkd", IOContext.DEFAULT)) {
- in.seek(indexFP);
- BKDReader r = new BKDReader(in);
+ try (IndexInput in = dir.openInput("bkd", IOContext.DEFAULT)) {
+ in.seek(indexFP);
+ BKDReader r = new BKDReader(in);
- int iters = atLeast(100);
- for(int iter=0;iter<iters;iter++) {
- if (VERBOSE) {
- System.out.println("\nTEST: iter=" + iter);
- }
+ int iters = atLeast(100);
+ for(int iter=0;iter<iters;iter++) {
+ if (VERBOSE) {
+ System.out.println("\nTEST: iter=" + iter);
+ }
- // Random N dims rect query:
- byte[][] queryMin = new byte[numDims][];
- byte[][] queryMax = new byte[numDims][];
- for(int dim=0;dim<numDims;dim++) {
- queryMin[dim] = new byte[numBytesPerDim];
- random().nextBytes(queryMin[dim]);
- queryMax[dim] = new byte[numBytesPerDim];
- random().nextBytes(queryMax[dim]);
- if (BKDUtil.compare(numBytesPerDim, queryMin[dim], 0, queryMax[dim], 0) > 0) {
- byte[] x = queryMin[dim];
- queryMin[dim] = queryMax[dim];
- queryMax[dim] = x;
- }
+ // Random N dims rect query:
+ byte[][] queryMin = new byte[numDims][];
+ byte[][] queryMax = new byte[numDims][];
+ for(int dim=0;dim<numDims;dim++) {
+ queryMin[dim] = new byte[numBytesPerDim];
+ random().nextBytes(queryMin[dim]);
+ queryMax[dim] = new byte[numBytesPerDim];
+ random().nextBytes(queryMax[dim]);
+ if (BKDUtil.compare(numBytesPerDim, queryMin[dim], 0, queryMax[dim], 0) > 0) {
+ byte[] x = queryMin[dim];
+ queryMin[dim] = queryMax[dim];
+ queryMax[dim] = x;
}
+ }
- final BitSet hits = new BitSet();
- r.intersect(new BKDReader.IntersectVisitor() {
+ final BitSet hits = new BitSet();
+ r.intersect(new BKDReader.IntersectVisitor() {
@Override
public void visit(int docID) {
hits.set(docID);
@@ -567,36 +647,37 @@ public class TestBKD extends LuceneTestC
}
});
- BitSet expected = new BitSet();
- for(int ord=0;ord<numValues;ord++) {
- boolean matches = true;
- for(int dim=0;dim<numDims;dim++) {
- byte[] x = docValues[ord][dim];
- if (BKDUtil.compare(numBytesPerDim, x, 0, queryMin[dim], 0) < 0 ||
- BKDUtil.compare(numBytesPerDim, x, 0, queryMax[dim], 0) > 0) {
- matches = false;
- break;
- }
+ BitSet expected = new BitSet();
+ for(int ord=0;ord<numValues;ord++) {
+ boolean matches = true;
+ for(int dim=0;dim<numDims;dim++) {
+ byte[] x = docValues[ord][dim];
+ if (BKDUtil.compare(numBytesPerDim, x, 0, queryMin[dim], 0) < 0 ||
+ BKDUtil.compare(numBytesPerDim, x, 0, queryMax[dim], 0) > 0) {
+ matches = false;
+ break;
}
+ }
- if (matches) {
- int docID;
- if (docIDs == null) {
- docID = ord;
- } else {
- docID = docIDs[ord];
- }
- expected.set(docID);
+ if (matches) {
+ int docID;
+ if (docIDs == null) {
+ docID = ord;
+ } else {
+ docID = docIDs[ord];
}
+ expected.set(docID);
}
+ }
- int limit = Math.max(expected.length(), hits.length());
- for(int docID=0;docID<limit;docID++) {
- assertEquals("docID=" + docID, expected.get(docID), hits.get(docID));
- }
+ int limit = Math.max(expected.length(), hits.length());
+ for(int docID=0;docID<limit;docID++) {
+ assertEquals("docID=" + docID, expected.get(docID), hits.get(docID));
}
}
}
+
+ dir.deleteFile("bkd");
}
private BigInteger randomBigInt(int numBytes) {
Modified: lucene/dev/branches/lucene6825/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java?rev=1709330&r1=1709329&r2=1709330&view=diff
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java (original)
+++ lucene/dev/branches/lucene6825/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java Sun Oct 18 23:15:44 2015
@@ -24,6 +24,7 @@ import java.util.Comparator;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.BytesRefIterator;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.OfflineSorter;
@@ -105,7 +106,7 @@ public class ExternalRefSorter implement
*/
class ByteSequenceIterator implements BytesRefIterator {
private final OfflineSorter.ByteSequencesReader reader;
- private BytesRef scratch = new BytesRef();
+ private BytesRefBuilder scratch = new BytesRefBuilder();
public ByteSequenceIterator(OfflineSorter.ByteSequencesReader reader) {
this.reader = reader;
@@ -118,17 +119,15 @@ public class ExternalRefSorter implement
}
boolean success = false;
try {
- byte[] next = reader.read();
- if (next != null) {
- scratch.bytes = next;
- scratch.length = next.length;
- scratch.offset = 0;
- } else {
+ if (reader.read(scratch) == false) {
IOUtils.close(reader);
scratch = null;
}
success = true;
- return scratch;
+ if (scratch == null) {
+ return null;
+ }
+ return scratch.get();
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(reader);
Modified: lucene/dev/branches/lucene6825/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java?rev=1709330&r1=1709329&r2=1709330&view=diff
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java (original)
+++ lucene/dev/branches/lucene6825/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java Sun Oct 18 23:15:44 2015
@@ -440,7 +440,6 @@ public class MockDirectoryWrapper extend
if (randomState.nextDouble() < randomIOExceptionRate) {
if (LuceneTestCase.VERBOSE) {
System.out.println(Thread.currentThread().getName() + ": MockDirectoryWrapper: now throw random exception" + (message == null ? "" : " (" + message + ")"));
- new Throwable().printStackTrace(System.out);
}
throw new IOException("a random IOException" + (message == null ? "" : " (" + message + ")"));
}
@@ -567,9 +566,6 @@ public class MockDirectoryWrapper extend
}
}
- if (crashed) {
- throw new IOException("cannot createOutput after crash");
- }
unSyncedFiles.add(name);
createdFiles.add(name);
@@ -607,6 +603,39 @@ public class MockDirectoryWrapper extend
}
}
+ @Override
+ public synchronized IndexOutput createTempOutput(String prefix, String suffix, IOContext context) throws IOException {
+ maybeThrowDeterministicException();
+ maybeThrowIOExceptionOnOpen("temp: prefix=" + prefix + " suffix=" + suffix);
+ maybeYield();
+ if (failOnCreateOutput) {
+ maybeThrowDeterministicException();
+ }
+ if (crashed) {
+ throw new IOException("cannot createTempOutput after crash");
+ }
+ init();
+
+ IndexOutput delegateOutput = in.createTempOutput(prefix, suffix, LuceneTestCase.newIOContext(randomState, context));
+ String name = delegateOutput.getName();
+ unSyncedFiles.add(name);
+ createdFiles.add(name);
+ final IndexOutput io = new MockIndexOutputWrapper(this, delegateOutput, name);
+ addFileHandle(io, name, Handle.Output);
+ openFilesForWrite.add(name);
+
+ // throttling REALLY slows down tests, so don't do it very often for SOMETIMES.
+ if (throttling == Throttling.ALWAYS ||
+ (throttling == Throttling.SOMETIMES && randomState.nextInt(200) == 0)) {
+ if (LuceneTestCase.VERBOSE) {
+ System.out.println("MockDirectoryWrapper: throttling indexOutput (" + name + ")");
+ }
+ return throttledOutput.newFromDelegate(io);
+ } else {
+ return io;
+ }
+ }
+
private static enum Handle {
Input, Output, Slice
}