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
   }