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/15 12:25:42 UTC

svn commit: r1708778 [4/8] - in /lucene/dev/branches/lucene6825: ./ lucene/ lucene/analysis/ lucene/analysis/common/ lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/ lucene/analysis/common/src/test/org/apache/lucene/analysis/core/ l...

Modified: lucene/dev/branches/lucene6825/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeWriter.java?rev=1708778&r1=1708777&r2=1708778&view=diff
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeWriter.java (original)
+++ lucene/dev/branches/lucene6825/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeWriter.java Thu Oct 15 10:25:39 2015
@@ -18,14 +18,13 @@ package org.apache.lucene.bkdtree;
  */
 
 import java.io.IOException;
-import java.nio.file.DirectoryStream;
-import java.nio.file.Files;
-import java.nio.file.Path;
 import java.util.Arrays;
 import java.util.Comparator;
 
 import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.ByteArrayDataOutput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
@@ -33,8 +32,8 @@ import org.apache.lucene.util.BytesRefBu
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.InPlaceMergeSorter;
 import org.apache.lucene.util.LongBitSet;
-import org.apache.lucene.util.OfflineSorter.ByteSequencesWriter;
 import org.apache.lucene.util.OfflineSorter;
+import org.apache.lucene.util.OfflineSorter.ByteSequencesWriter;
 import org.apache.lucene.util.RamUsageEstimator;
 
 // TODO
@@ -88,22 +87,27 @@ class BKDTreeWriter {
   private final byte[] scratchBytes = new byte[BYTES_PER_DOC];
   private final ByteArrayDataOutput scratchBytesOutput = new ByteArrayDataOutput(scratchBytes);
 
-  private OfflineSorter.ByteSequencesWriter writer;
+  private final Directory tempDir;
+  private final String tempFileNamePrefix;
+
+  private OfflineSorter.ByteSequencesWriter offlineWriter;
   private GrowingHeapLatLonWriter heapWriter;
 
-  private Path tempInput;
+  private IndexOutput tempInput;
   private final int maxPointsInLeafNode;
   private final int maxPointsSortInHeap;
 
   private long pointCount;
 
-  public BKDTreeWriter() throws IOException {
-    this(DEFAULT_MAX_POINTS_IN_LEAF_NODE, DEFAULT_MAX_POINTS_SORT_IN_HEAP);
+  public BKDTreeWriter(Directory tempDir, String tempFileNamePrefix) throws IOException {
+    this(tempDir, tempFileNamePrefix, DEFAULT_MAX_POINTS_IN_LEAF_NODE, DEFAULT_MAX_POINTS_SORT_IN_HEAP);
   }
 
   // TODO: instead of maxPointsSortInHeap, change to maxMBHeap ... the mapping is non-obvious:
-  public BKDTreeWriter(int maxPointsInLeafNode, int maxPointsSortInHeap) throws IOException {
+  public BKDTreeWriter(Directory tempDir, String tempFileNamePrefix, int maxPointsInLeafNode, int maxPointsSortInHeap) throws IOException {
     verifyParams(maxPointsInLeafNode, maxPointsSortInHeap);
+    this.tempDir = tempDir;
+    this.tempFileNamePrefix = tempFileNamePrefix;
     this.maxPointsInLeafNode = maxPointsInLeafNode;
     this.maxPointsSortInHeap = maxPointsSortInHeap;
 
@@ -143,8 +147,8 @@ class BKDTreeWriter {
   private void switchToOffline() throws IOException {
 
     // For each .add we just append to this input file, then in .finish we sort this input and resursively build the tree:
-    tempInput = Files.createTempFile(OfflineSorter.getDefaultTempDir(), "in", "");
-    writer = new OfflineSorter.ByteSequencesWriter(tempInput);
+    tempInput = tempDir.createTempOutput(tempFileNamePrefix, "bkd", IOContext.DEFAULT);
+    offlineWriter = new OfflineSorter.ByteSequencesWriter(tempInput);
     for(int i=0;i<pointCount;i++) {
       scratchBytesOutput.reset(scratchBytes);
       scratchBytesOutput.writeInt(heapWriter.latEncs[i]);
@@ -152,7 +156,7 @@ class BKDTreeWriter {
       scratchBytesOutput.writeVInt(heapWriter.docIDs[i]);
       scratchBytesOutput.writeVLong(i);
       // TODO: can/should OfflineSorter optimize the fixed-width case?
-      writer.write(scratchBytes, 0, scratchBytes.length);
+      offlineWriter.write(scratchBytes, 0, scratchBytes.length);
     }
 
     heapWriter = null;
@@ -165,7 +169,7 @@ class BKDTreeWriter {
     assert lonEnc < Integer.MAX_VALUE;
 
     if (pointCount >= maxPointsSortInHeap) {
-      if (writer == null) {
+      if (offlineWriter == null) {
         switchToOffline();
       }
       scratchBytesOutput.reset(scratchBytes);
@@ -173,7 +177,7 @@ class BKDTreeWriter {
       scratchBytesOutput.writeInt(lonEnc);
       scratchBytesOutput.writeVInt(docID);
       scratchBytesOutput.writeVLong(pointCount);
-      writer.write(scratchBytes, 0, scratchBytes.length);
+      offlineWriter.write(scratchBytes, 0, scratchBytes.length);
     } else {
       // Not too many points added yet, continue using heap:
       heapWriter.append(latEnc, lonEnc, pointCount, docID);
@@ -184,7 +188,7 @@ class BKDTreeWriter {
 
   /** 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 LatLonWriter convertToFixedWidth(Path in) throws IOException {
+  private LatLonWriter convertToFixedWidth(String in) throws IOException {
     BytesRefBuilder scratch = new BytesRefBuilder();
     scratch.grow(BYTES_PER_DOC);
     BytesRef bytes = scratch.get();
@@ -194,7 +198,7 @@ class BKDTreeWriter {
     LatLonWriter sortedWriter = null;
     boolean success = false;
     try {
-      reader = new OfflineSorter.ByteSequencesReader(in);
+      reader = new OfflineSorter.ByteSequencesReader(tempDir.openInput(in, IOContext.READONCE));
       sortedWriter = getWriter(pointCount);
       for (long i=0;i<pointCount;i++) {
         boolean result = reader.read(scratch);
@@ -230,10 +234,10 @@ class BKDTreeWriter {
 
   private LatLonWriter sort(boolean lon) throws IOException {
     if (heapWriter != null) {
+      // All buffered points are still in heap
 
       assert pointCount < Integer.MAX_VALUE;
 
-      // All buffered points are still in heap
       new InPlaceMergeSorter() {
         @Override
         protected void swap(int i, int j) {
@@ -329,20 +333,19 @@ class BKDTreeWriter {
         }
       };
 
-      Path sorted = Files.createTempFile(OfflineSorter.getDefaultTempDir(), "sorted", "");
-      boolean success = false;
       
+      boolean success = false;
+      OfflineSorter sorter = new OfflineSorter(tempDir, tempFileNamePrefix, cmp);
+      String sortedFileName = sorter.sort(tempInput.getName());
       try {
-        OfflineSorter latSorter = new OfflineSorter(cmp);
-        latSorter.sort(tempInput, sorted);
-        LatLonWriter writer = convertToFixedWidth(sorted);
+        LatLonWriter writer = convertToFixedWidth(sortedFileName);
         success = true;
         return writer;
       } finally {
         if (success) {
-          IOUtils.rm(sorted);
+          tempDir.deleteFile(sortedFileName);
         } else {
-          IOUtils.deleteFilesIgnoringExceptions(sorted);
+          IOUtils.deleteFilesIgnoringExceptions(tempDir, sortedFileName);
         }
       }
     }
@@ -352,8 +355,9 @@ class BKDTreeWriter {
   public long finish(IndexOutput out) throws IOException {
     //System.out.println("\nBKDTreeWriter.finish pointCount=" + pointCount + " out=" + out + " heapWriter=" + heapWriter);
 
-    if (writer != null) {
-      writer.close();
+    if (offlineWriter != null) {
+      // This also closes the temp file output:
+      offlineWriter.close();
     }
 
     LongBitSet bitSet = new LongBitSet(pointCount);
@@ -410,7 +414,9 @@ class BKDTreeWriter {
       if (success) {
         latSortedWriter.destroy();
         lonSortedWriter.destroy();
-        IOUtils.rm(tempInput);
+        if (tempInput != null) {
+          tempDir.deleteFile(tempInput.getName());
+        }
       } else {
         try {
           latSortedWriter.destroy();
@@ -422,7 +428,9 @@ class BKDTreeWriter {
         } catch (Throwable t) {
           // Suppress to keep throwing original exc
         }
-        IOUtils.deleteFilesIgnoringExceptions(tempInput);
+        if (tempInput != null) {
+          IOUtils.deleteFilesIgnoringExceptions(tempDir, tempInput.getName());
+        }
       }
     }
 
@@ -813,7 +821,7 @@ class BKDTreeWriter {
     if (count < maxPointsSortInHeap) {
       return new HeapLatLonWriter((int) count);
     } else {
-      return new OfflineLatLonWriter(count);
+      return new OfflineLatLonWriter(tempDir, tempFileNamePrefix, count);
     }
   }
 

Modified: lucene/dev/branches/lucene6825/lucene/sandbox/src/java/org/apache/lucene/bkdtree/OfflineLatLonReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/sandbox/src/java/org/apache/lucene/bkdtree/OfflineLatLonReader.java?rev=1708778&r1=1708777&r2=1708778&view=diff
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/sandbox/src/java/org/apache/lucene/bkdtree/OfflineLatLonReader.java (original)
+++ lucene/dev/branches/lucene6825/lucene/sandbox/src/java/org/apache/lucene/bkdtree/OfflineLatLonReader.java Thu Oct 15 10:25:39 2015
@@ -17,34 +17,23 @@ package org.apache.lucene.bkdtree;
  * limitations under the License.
  */
 
-import java.io.BufferedInputStream;
 import java.io.IOException;
-import java.io.InputStream;
-import java.nio.file.Files;
-import java.nio.file.Path;
 
-import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
 
 final class OfflineLatLonReader implements LatLonReader {
-  final InputStreamDataInput in;
+  final IndexInput in;
   long countLeft;
   private int latEnc;
   private int lonEnc;
   private long ord;
   private int docID;
 
-  OfflineLatLonReader(Path tempFile, long start, long count) throws IOException {
-    InputStream fis = Files.newInputStream(tempFile);
-    long seekFP = start * BKDTreeWriter.BYTES_PER_DOC;
-    long skipped = 0;
-    while (skipped < seekFP) {
-      long inc = fis.skip(seekFP - skipped);
-      skipped += inc;
-      if (inc == 0) {
-        throw new RuntimeException("skip returned 0");
-      }
-    }
-    in = new InputStreamDataInput(new BufferedInputStream(fis));
+  OfflineLatLonReader(Directory tempDir, String tempFileName, long start, long count) throws IOException {
+    in = tempDir.openInput(tempFileName, IOContext.READONCE);
+    in.seek(start * BKDTreeWriter.BYTES_PER_DOC);
     this.countLeft = count;
   }
 

Modified: lucene/dev/branches/lucene6825/lucene/sandbox/src/java/org/apache/lucene/bkdtree/OfflineLatLonWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/sandbox/src/java/org/apache/lucene/bkdtree/OfflineLatLonWriter.java?rev=1708778&r1=1708777&r2=1708778&view=diff
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/sandbox/src/java/org/apache/lucene/bkdtree/OfflineLatLonWriter.java (original)
+++ lucene/dev/branches/lucene6825/lucene/sandbox/src/java/org/apache/lucene/bkdtree/OfflineLatLonWriter.java Thu Oct 15 10:25:39 2015
@@ -17,29 +17,26 @@ package org.apache.lucene.bkdtree;
  * limitations under the License.
  */
 
-import org.apache.lucene.store.ByteArrayDataOutput;
-import org.apache.lucene.store.OutputStreamDataOutput;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.OfflineSorter;
-
-import 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;
 
 final class OfflineLatLonWriter implements LatLonWriter {
 
-  final Path tempFile;
+  final Directory tempDir;
   final byte[] scratchBytes = new byte[BKDTreeWriter.BYTES_PER_DOC];
   final ByteArrayDataOutput scratchBytesOutput = new ByteArrayDataOutput(scratchBytes);      
-  final OutputStreamDataOutput out;
+  final IndexOutput out;
   final long count;
   private long countWritten;
   private boolean closed;
 
-  public OfflineLatLonWriter(long count) throws IOException {
-    tempFile = Files.createTempFile(OfflineSorter.getDefaultTempDir(), "size" + count + ".", "");
-    out = new OutputStreamDataOutput(new BufferedOutputStream(Files.newOutputStream(tempFile)));
+  public OfflineLatLonWriter(Directory tempDir, String tempFileNamePrefix, long count) throws IOException {
+    this.tempDir = tempDir;
+    out = tempDir.createTempOutput(tempFileNamePrefix, "bkd", IOContext.DEFAULT);
     this.count = count;
   }
     
@@ -55,7 +52,7 @@ final class OfflineLatLonWriter implemen
   @Override
   public LatLonReader getReader(long start) throws IOException {
     assert closed;
-    return new OfflineLatLonReader(tempFile, start, count-start);
+    return new OfflineLatLonReader(tempDir, out.getName(), start, count-start);
   }
 
   @Override
@@ -69,12 +66,12 @@ final class OfflineLatLonWriter implemen
 
   @Override
   public void destroy() throws IOException {
-    IOUtils.rm(tempFile);
+    tempDir.deleteFile(out.getName());
   }
 
   @Override
   public String toString() {
-    return "OfflineLatLonWriter(count=" + count + " tempFile=" + tempFile + ")";
+    return "OfflineLatLonWriter(count=" + count + " tempFileName=" + out.getName() + ")";
   }
 }
 

Modified: lucene/dev/branches/lucene6825/lucene/sandbox/src/java/org/apache/lucene/rangetree/OfflineSliceReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/sandbox/src/java/org/apache/lucene/rangetree/OfflineSliceReader.java?rev=1708778&r1=1708777&r2=1708778&view=diff
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/sandbox/src/java/org/apache/lucene/rangetree/OfflineSliceReader.java (original)
+++ lucene/dev/branches/lucene6825/lucene/sandbox/src/java/org/apache/lucene/rangetree/OfflineSliceReader.java Thu Oct 15 10:25:39 2015
@@ -17,33 +17,22 @@ package org.apache.lucene.rangetree;
  * limitations under the License.
  */
 
-import java.io.BufferedInputStream;
 import java.io.IOException;
-import java.io.InputStream;
-import java.nio.file.Files;
-import java.nio.file.Path;
 
-import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
 
 final class OfflineSliceReader implements SliceReader {
-  final InputStreamDataInput in;
-  long countLeft;
+  final IndexInput in;
+  private long countLeft;
   private long value;
   private long ord;
   private int docID;
 
-  OfflineSliceReader(Path tempFile, long start, long count) throws IOException {
-    InputStream fis = Files.newInputStream(tempFile);
-    long seekFP = start * RangeTreeWriter.BYTES_PER_DOC;
-    long skipped = 0;
-    while (skipped < seekFP) {
-      long inc = fis.skip(seekFP - skipped);
-      skipped += inc;
-      if (inc == 0) {
-        throw new RuntimeException("skip returned 0");
-      }
-    }
-    in = new InputStreamDataInput(new BufferedInputStream(fis));
+  OfflineSliceReader(Directory tempDir, String tempFileName, long start, long count) throws IOException {
+    in = tempDir.openInput(tempFileName, IOContext.READONCE);
+    in.seek(start * RangeTreeWriter.BYTES_PER_DOC);
     this.countLeft = count;
   }
 

Modified: lucene/dev/branches/lucene6825/lucene/sandbox/src/java/org/apache/lucene/rangetree/OfflineSliceWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/sandbox/src/java/org/apache/lucene/rangetree/OfflineSliceWriter.java?rev=1708778&r1=1708777&r2=1708778&view=diff
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/sandbox/src/java/org/apache/lucene/rangetree/OfflineSliceWriter.java (original)
+++ lucene/dev/branches/lucene6825/lucene/sandbox/src/java/org/apache/lucene/rangetree/OfflineSliceWriter.java Thu Oct 15 10:25:39 2015
@@ -17,50 +17,47 @@ package org.apache.lucene.rangetree;
  * limitations under the License.
  */
 
-import org.apache.lucene.store.ByteArrayDataOutput;
-import org.apache.lucene.store.OutputStreamDataOutput;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.OfflineSorter;
-
-import 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;
 
 final class OfflineSliceWriter implements SliceWriter {
 
-  final Path tempFile;
+  final Directory tempDir;
   final byte[] scratchBytes = new byte[RangeTreeWriter.BYTES_PER_DOC];
   final ByteArrayDataOutput scratchBytesOutput = new ByteArrayDataOutput(scratchBytes);      
-  final OutputStreamDataOutput out;
+  final IndexOutput tempFile;
   final long count;
   private boolean closed;
   private long countWritten;
 
-  public OfflineSliceWriter(long count) throws IOException {
-    tempFile = Files.createTempFile(OfflineSorter.getDefaultTempDir(), "size" + count + ".", "");
-    out = new OutputStreamDataOutput(new BufferedOutputStream(Files.newOutputStream(tempFile)));
+  public OfflineSliceWriter(Directory tempDir, String tempFileNamePrefix, long count) throws IOException {
+    this.tempDir = tempDir;
+    tempFile = tempDir.createTempOutput(tempFileNamePrefix, "rangetree", IOContext.DEFAULT);
     this.count = count;
   }
     
   @Override
   public void append(long value, long ord, int docID) throws IOException {
-    out.writeLong(value);
-    out.writeLong(ord);
-    out.writeInt(docID);
+    tempFile.writeLong(value);
+    tempFile.writeLong(ord);
+    tempFile.writeInt(docID);
     countWritten++;
   }
 
   @Override
   public SliceReader getReader(long start) throws IOException {
     assert closed;
-    return new OfflineSliceReader(tempFile, start, count-start);
+    return new OfflineSliceReader(tempDir, tempFile.getName(), start, count-start);
   }
 
   @Override
   public void close() throws IOException {
     closed = true;
-    out.close();
+    tempFile.close();
     if (count != countWritten) {
       throw new IllegalStateException("wrote " + countWritten + " values, but expected " + count);
     }
@@ -68,12 +65,12 @@ final class OfflineSliceWriter implement
 
   @Override
   public void destroy() throws IOException {
-    IOUtils.rm(tempFile);
+    tempDir.deleteFile(tempFile.getName());
   }
 
   @Override
   public String toString() {
-    return "OfflineSliceWriter(count=" + count + " tempFile=" + tempFile + ")";
+    return "OfflineSliceWriter(count=" + count + " tempFileName=" + tempFile.getName() + ")";
   }
 }
 

Modified: lucene/dev/branches/lucene6825/lucene/sandbox/src/java/org/apache/lucene/rangetree/RangeTreeDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/sandbox/src/java/org/apache/lucene/rangetree/RangeTreeDocValuesConsumer.java?rev=1708778&r1=1708777&r2=1708778&view=diff
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/sandbox/src/java/org/apache/lucene/rangetree/RangeTreeDocValuesConsumer.java (original)
+++ lucene/dev/branches/lucene6825/lucene/sandbox/src/java/org/apache/lucene/rangetree/RangeTreeDocValuesConsumer.java Thu Oct 15 10:25:39 2015
@@ -91,7 +91,7 @@ class RangeTreeDocValuesConsumer extends
   @Override
   public void addSortedNumericField(FieldInfo field, Iterable<Number> docToValueCount, Iterable<Number> values) throws IOException {
     delegate.addSortedNumericField(field, docToValueCount, values);
-    RangeTreeWriter writer = new RangeTreeWriter(maxPointsInLeafNode, maxPointsSortInHeap);
+    RangeTreeWriter writer = new RangeTreeWriter(state.directory, state.segmentInfo.name, maxPointsInLeafNode, maxPointsSortInHeap);
     Iterator<Number> valueIt = values.iterator();
     Iterator<Number> valueCountIt = docToValueCount.iterator();
     //System.out.println("\nSNF: field=" + field.name);
@@ -127,7 +127,7 @@ class RangeTreeDocValuesConsumer extends
   @Override
   public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrdCount, Iterable<Number> ords) throws IOException {
     delegate.addSortedSetField(field, values, docToOrdCount, ords);
-    RangeTreeWriter writer = new RangeTreeWriter(maxPointsInLeafNode, maxPointsSortInHeap);
+    RangeTreeWriter writer = new RangeTreeWriter(state.directory, state.segmentInfo.name, maxPointsInLeafNode, maxPointsSortInHeap);
     Iterator<Number> docToOrdCountIt = docToOrdCount.iterator();
     Iterator<Number> ordsIt = ords.iterator();
     //System.out.println("\nSSF: field=" + field.name);

Modified: lucene/dev/branches/lucene6825/lucene/sandbox/src/java/org/apache/lucene/rangetree/RangeTreeWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/sandbox/src/java/org/apache/lucene/rangetree/RangeTreeWriter.java?rev=1708778&r1=1708777&r2=1708778&view=diff
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/sandbox/src/java/org/apache/lucene/rangetree/RangeTreeWriter.java (original)
+++ lucene/dev/branches/lucene6825/lucene/sandbox/src/java/org/apache/lucene/rangetree/RangeTreeWriter.java Thu Oct 15 10:25:39 2015
@@ -18,22 +18,21 @@ package org.apache.lucene.rangetree;
  */
 
 import java.io.IOException;
-import java.nio.file.DirectoryStream;
-import java.nio.file.Files;
-import java.nio.file.Path;
 import java.util.Arrays;
 import java.util.Comparator;
 
 import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.ByteArrayDataOutput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
 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.OfflineSorter.ByteSequencesWriter;
 import org.apache.lucene.util.OfflineSorter;
+import org.apache.lucene.util.OfflineSorter.ByteSequencesWriter;
 import org.apache.lucene.util.RamUsageEstimator;
 
 // TODO
@@ -77,10 +76,13 @@ class RangeTreeWriter {
   private final byte[] scratchBytes = new byte[BYTES_PER_DOC];
   private final ByteArrayDataOutput scratchBytesOutput = new ByteArrayDataOutput(scratchBytes);
 
-  private OfflineSorter.ByteSequencesWriter writer;
+  private final Directory tempDir;
+  private final String tempFileNamePrefix;
+
+  private OfflineSorter.ByteSequencesWriter offlineWriter;
   private GrowingHeapSliceWriter heapWriter;
 
-  private Path tempInput;
+  private IndexOutput tempInput;
   private final int maxValuesInLeafNode;
   private final int maxValuesSortInHeap;
 
@@ -88,13 +90,15 @@ class RangeTreeWriter {
   private long globalMinValue = Long.MAX_VALUE;
   private long globalMaxValue = Long.MIN_VALUE;
 
-  public RangeTreeWriter() throws IOException {
-    this(DEFAULT_MAX_VALUES_IN_LEAF_NODE, DEFAULT_MAX_VALUES_SORT_IN_HEAP);
+  public RangeTreeWriter(Directory tempDir, String tempFileNamePrefix) throws IOException {
+    this(tempDir, tempFileNamePrefix, DEFAULT_MAX_VALUES_IN_LEAF_NODE, DEFAULT_MAX_VALUES_SORT_IN_HEAP);
   }
 
   // TODO: instead of maxValuesSortInHeap, change to maxMBHeap ... the mapping is non-obvious:
-  public RangeTreeWriter(int maxValuesInLeafNode, int maxValuesSortInHeap) throws IOException {
+  public RangeTreeWriter(Directory tempDir, String tempFileNamePrefix, int maxValuesInLeafNode, int maxValuesSortInHeap) throws IOException {
     verifyParams(maxValuesInLeafNode, maxValuesSortInHeap);
+    this.tempDir = tempDir;
+    this.tempFileNamePrefix = tempFileNamePrefix;
     this.maxValuesInLeafNode = maxValuesInLeafNode;
     this.maxValuesSortInHeap = maxValuesSortInHeap;
 
@@ -121,15 +125,15 @@ class RangeTreeWriter {
   private void switchToOffline() throws IOException {
 
     // For each .add we just append to this input file, then in .finish we sort this input and resursively build the tree:
-    tempInput = Files.createTempFile(OfflineSorter.getDefaultTempDir(), "in", "");
-    writer = new OfflineSorter.ByteSequencesWriter(tempInput);
+    tempInput = tempDir.createTempOutput(tempFileNamePrefix, "rangetree", IOContext.DEFAULT);
+    offlineWriter = new OfflineSorter.ByteSequencesWriter(tempInput);
     for(int i=0;i<valueCount;i++) {
       scratchBytesOutput.reset(scratchBytes);
       scratchBytesOutput.writeLong(heapWriter.values[i]);
       scratchBytesOutput.writeVInt(heapWriter.docIDs[i]);
       scratchBytesOutput.writeVLong(i);
       // TODO: can/should OfflineSorter optimize the fixed-width case?
-      writer.write(scratchBytes, 0, scratchBytes.length);
+      offlineWriter.write(scratchBytes, 0, scratchBytes.length);
     }
 
     heapWriter = null;
@@ -137,14 +141,14 @@ class RangeTreeWriter {
 
   void add(long value, int docID) throws IOException {
     if (valueCount >= maxValuesSortInHeap) {
-      if (writer == null) {
+      if (offlineWriter == null) {
         switchToOffline();
       }
       scratchBytesOutput.reset(scratchBytes);
       scratchBytesOutput.writeLong(value);
       scratchBytesOutput.writeVInt(docID);
       scratchBytesOutput.writeVLong(valueCount);
-      writer.write(scratchBytes, 0, scratchBytes.length);
+      offlineWriter.write(scratchBytes, 0, scratchBytes.length);
     } else {
       // Not too many points added yet, continue using heap:
       heapWriter.append(value, valueCount, docID);
@@ -157,7 +161,7 @@ class RangeTreeWriter {
 
   /** 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 SliceWriter convertToFixedWidth(Path in) throws IOException {
+  private SliceWriter convertToFixedWidth(String in) throws IOException {
     BytesRefBuilder scratch = new BytesRefBuilder();
     scratch.grow(BYTES_PER_DOC);
     BytesRef bytes = scratch.get();
@@ -167,7 +171,7 @@ class RangeTreeWriter {
     SliceWriter sortedWriter = null;
     boolean success = false;
     try {
-      reader = new OfflineSorter.ByteSequencesReader(in);
+      reader = new OfflineSorter.ByteSequencesReader(tempDir.openInput(in, IOContext.READONCE));
       sortedWriter = getWriter(valueCount);
       for (long i=0;i<valueCount;i++) {
         boolean result = reader.read(scratch);
@@ -280,19 +284,18 @@ class RangeTreeWriter {
         }
       };
 
-      Path sorted = Files.createTempFile(OfflineSorter.getDefaultTempDir(), "sorted", "");
       boolean success = false;
+      OfflineSorter sorter = new OfflineSorter(tempDir, tempFileNamePrefix, cmp);
+      String sortedFileName = sorter.sort(tempInput.getName());
       try {
-        OfflineSorter sorter = new OfflineSorter(cmp);
-        sorter.sort(tempInput, sorted);
-        SliceWriter writer = convertToFixedWidth(sorted);
+        SliceWriter writer = convertToFixedWidth(sortedFileName);
         success = true;
         return writer;
       } finally {
         if (success) {
-          IOUtils.rm(sorted);
+          tempDir.deleteFile(sortedFileName);
         } else {
-          IOUtils.deleteFilesIgnoringExceptions(sorted);
+          IOUtils.deleteFilesIgnoringExceptions(tempDir, sortedFileName);
         }
       }
     }
@@ -301,8 +304,8 @@ class RangeTreeWriter {
   /** Writes the 1d BKD tree to the provided {@link IndexOutput} and returns the file offset where index was written. */
   public long finish(IndexOutput out) throws IOException {
 
-    if (writer != null) {
-      writer.close();
+    if (offlineWriter != null) {
+      offlineWriter.close();
     }
 
     if (valueCount == 0) {
@@ -357,14 +360,18 @@ class RangeTreeWriter {
     } finally {
       if (success) {
         sortedWriter.destroy();
-        IOUtils.rm(tempInput);
+        if (tempInput != null) {
+          tempDir.deleteFile(tempInput.getName());
+        }
       } else {
         try {
           sortedWriter.destroy();
         } catch (Throwable t) {
           // Suppress to keep throwing original exc
         }
-        IOUtils.deleteFilesIgnoringExceptions(tempInput);
+        if (tempInput != null) {
+          IOUtils.deleteFilesIgnoringExceptions(tempDir, tempInput.getName());
+        }
       }
     }
 
@@ -567,7 +574,7 @@ class RangeTreeWriter {
     if (count < maxValuesSortInHeap) {
       return new HeapSliceWriter((int) count);
     } else {
-      return new OfflineSliceWriter(count);
+      return new OfflineSliceWriter(tempDir, tempFileNamePrefix, count);
     }
   }
 }

Modified: lucene/dev/branches/lucene6825/lucene/sandbox/src/test/org/apache/lucene/bkdtree/TestBKDTree.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/sandbox/src/test/org/apache/lucene/bkdtree/TestBKDTree.java?rev=1708778&r1=1708777&r2=1708778&view=diff
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/sandbox/src/test/org/apache/lucene/bkdtree/TestBKDTree.java (original)
+++ lucene/dev/branches/lucene6825/lucene/sandbox/src/test/org/apache/lucene/bkdtree/TestBKDTree.java Thu Oct 15 10:25:39 2015
@@ -17,6 +17,15 @@ package org.apache.lucene.bkdtree;
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
+
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.lucene53.Lucene53Codec;
@@ -37,24 +46,16 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.SimpleCollector;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.LuceneTestCase.Nightly;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.LuceneTestCase.Nightly;
 import org.apache.lucene.util.TestUtil;
 import org.junit.BeforeClass;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicBoolean;
-
 public class TestBKDTree extends LuceneTestCase {
 
   private static boolean smallBBox;
@@ -163,7 +164,7 @@ public class TestBKDTree extends LuceneT
     // Every doc has 2 points:
     double[] lats = new double[2*numPoints];
     double[] lons = new double[2*numPoints];
-    Directory dir = newDirectory();
+    Directory dir = getDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig();
     // We rely on docID order:
     iwc.setMergePolicy(newLogMergePolicy());
@@ -378,7 +379,7 @@ public class TestBKDTree extends LuceneT
     if (lats.length > 100000) {
       dir = newFSDirectory(createTempDir("TestBKDTree"));
     } else {
-      dir = newDirectory();
+      dir = getDirectory();
     }
     Set<Integer> deleted = new HashSet<>();
     // RandomIndexWriter is too slow here:
@@ -608,7 +609,7 @@ public class TestBKDTree extends LuceneT
   }
 
   public void testAccountableHasDelegate() throws Exception {
-    Directory dir = newDirectory();
+    Directory dir = getDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig();
     Codec codec = TestUtil.alwaysDocValuesFormat(getDocValuesFormat());
     iwc.setCodec(codec);
@@ -632,4 +633,12 @@ public class TestBKDTree extends LuceneT
     int maxPointsSortInHeap = TestUtil.nextInt(random(), maxPointsInLeaf, 1024*1024);
     return new BKDTreeDocValuesFormat(maxPointsInLeaf, maxPointsSortInHeap);
   }
+
+  private static Directory getDirectory() {     
+    Directory dir = newDirectory();
+    if (dir instanceof MockDirectoryWrapper) {
+      ((MockDirectoryWrapper) dir).setEnableVirusScanner(false);
+    }
+    return dir;
+  }
 }

Modified: lucene/dev/branches/lucene6825/lucene/sandbox/src/test/org/apache/lucene/rangetree/TestRangeTree.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/sandbox/src/test/org/apache/lucene/rangetree/TestRangeTree.java?rev=1708778&r1=1708777&r2=1708778&view=diff
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/sandbox/src/test/org/apache/lucene/rangetree/TestRangeTree.java (original)
+++ lucene/dev/branches/lucene6825/lucene/sandbox/src/test/org/apache/lucene/rangetree/TestRangeTree.java Thu Oct 15 10:25:39 2015
@@ -17,6 +17,14 @@ package org.apache.lucene.rangetree;
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
+
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.lucene53.Lucene53Codec;
@@ -39,6 +47,7 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.SimpleCollector;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.Bits;
@@ -49,14 +58,6 @@ import org.apache.lucene.util.LuceneTest
 import org.apache.lucene.util.TestUtil;
 import org.junit.BeforeClass;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicBoolean;
-
 public class TestRangeTree extends LuceneTestCase {
 
   // Controls what range of values we randomly generate, so we sometimes test narrow ranges:
@@ -116,7 +117,7 @@ public class TestRangeTree extends Lucen
     int numValues = atLeast(10000);
     // Every doc has 2 values:
     long[] values = new long[2*numValues];
-    Directory dir = newDirectory();
+    Directory dir = getDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig();
 
     // We rely on docID order:
@@ -201,7 +202,7 @@ public class TestRangeTree extends Lucen
     int numValues = atLeast(10000);
     // Every doc has 2 values:
     long[] values = new long[2*numValues];
-    Directory dir = newDirectory();
+    Directory dir = getDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig();
 
     // We rely on docID order:
@@ -370,7 +371,7 @@ public class TestRangeTree extends Lucen
     if (values.length > 100000) {
       dir = newFSDirectory(createTempDir("TestRangeTree"));
     } else {
-      dir = newDirectory();
+      dir = getDirectory();
     }
     Set<Integer> deleted = new HashSet<>();
     // RandomIndexWriter is too slow here:
@@ -534,7 +535,7 @@ public class TestRangeTree extends Lucen
   }
 
   public void testAccountableHasDelegate() throws Exception {
-    Directory dir = newDirectory();
+    Directory dir = getDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig();
     Codec codec = TestUtil.alwaysDocValuesFormat(new RangeTreeDocValuesFormat());
     iwc.setCodec(codec);
@@ -554,7 +555,7 @@ public class TestRangeTree extends Lucen
   }
 
   public void testMinMaxLong() throws Exception {
-    Directory dir = newDirectory();
+    Directory dir = getDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig();
     Codec codec = TestUtil.alwaysDocValuesFormat(new RangeTreeDocValuesFormat());
     iwc.setCodec(codec);
@@ -580,7 +581,7 @@ public class TestRangeTree extends Lucen
   }
 
   public void testBasicSortedSet() throws Exception {
-    Directory dir = newDirectory();
+    Directory dir = getDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig();
     Codec codec = TestUtil.alwaysDocValuesFormat(new RangeTreeDocValuesFormat());
     iwc.setCodec(codec);
@@ -613,7 +614,7 @@ public class TestRangeTree extends Lucen
   }
 
   public void testLongMinMaxNumeric() throws Exception {
-    Directory dir = newDirectory();
+    Directory dir = getDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig();
     Codec codec = TestUtil.alwaysDocValuesFormat(new RangeTreeDocValuesFormat());
     iwc.setCodec(codec);
@@ -641,7 +642,7 @@ public class TestRangeTree extends Lucen
   }
 
   public void testLongMinMaxSortedSet() throws Exception {
-    Directory dir = newDirectory();
+    Directory dir = getDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig();
     Codec codec = TestUtil.alwaysDocValuesFormat(new RangeTreeDocValuesFormat());
     iwc.setCodec(codec);
@@ -669,7 +670,7 @@ public class TestRangeTree extends Lucen
   }
 
   public void testSortedSetNoOrdsMatch() throws Exception {
-    Directory dir = newDirectory();
+    Directory dir = getDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig();
     Codec codec = TestUtil.alwaysDocValuesFormat(new RangeTreeDocValuesFormat());
     iwc.setCodec(codec);
@@ -693,7 +694,7 @@ public class TestRangeTree extends Lucen
   }
 
   public void testNumericNoValuesMatch() throws Exception {
-    Directory dir = newDirectory();
+    Directory dir = getDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig();
     Codec codec = TestUtil.alwaysDocValuesFormat(new RangeTreeDocValuesFormat());
     iwc.setCodec(codec);
@@ -715,7 +716,7 @@ public class TestRangeTree extends Lucen
   }
 
   public void testNoDocs() throws Exception {
-    Directory dir = newDirectory();
+    Directory dir = getDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig();
     Codec codec = TestUtil.alwaysDocValuesFormat(new RangeTreeDocValuesFormat());
     iwc.setCodec(codec);
@@ -766,4 +767,12 @@ public class TestRangeTree extends Lucen
     int maxPointsSortInHeap = TestUtil.nextInt(random(), maxPointsInLeaf, 1024*1024);
     return new RangeTreeDocValuesFormat(maxPointsInLeaf, maxPointsSortInHeap);
   }
+
+  private static Directory getDirectory() {     
+    Directory dir = newDirectory();
+    if (dir instanceof MockDirectoryWrapper) {
+      ((MockDirectoryWrapper) dir).setEnableVirusScanner(false);
+    }
+    return dir;
+  }
 }

Modified: lucene/dev/branches/lucene6825/lucene/sandbox/src/test/org/apache/lucene/search/TestTermAutomatonQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/sandbox/src/test/org/apache/lucene/search/TestTermAutomatonQuery.java?rev=1708778&r1=1708777&r2=1708778&view=diff
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/sandbox/src/test/org/apache/lucene/search/TestTermAutomatonQuery.java (original)
+++ lucene/dev/branches/lucene6825/lucene/sandbox/src/test/org/apache/lucene/search/TestTermAutomatonQuery.java Thu Oct 15 10:25:39 2015
@@ -47,6 +47,7 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BitDocIdSet;
+import org.apache.lucene.util.BitSetIterator;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;
@@ -584,7 +585,7 @@ public class TestTermAutomatonQuery exte
         if (VERBOSE) {
           System.out.println("  use random filter");
         }
-        RandomFilter filter = new RandomFilter(random().nextLong(), random().nextFloat());
+        RandomQuery filter = new RandomQuery(random().nextLong(), random().nextFloat());
         q1 = new BooleanQuery.Builder()
             .add(q1, Occur.MUST)
             .add(filter, Occur.FILTER)
@@ -630,29 +631,33 @@ public class TestTermAutomatonQuery exte
     return result;
   }
 
-  private static class RandomFilter extends Filter {
+  private static class RandomQuery extends Query {
     private final long seed;
     private float density;
 
     // density should be 0.0 ... 1.0
-    public RandomFilter(long seed, float density) {
+    public RandomQuery(long seed, float density) {
       this.seed = seed;
       this.density = density;
     }
 
     @Override
-    public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
-      int maxDoc = context.reader().maxDoc();
-      FixedBitSet bits = new FixedBitSet(maxDoc);
-      Random random = new Random(seed ^ context.docBase);
-      for(int docID=0;docID<maxDoc;docID++) {
-        if (random.nextFloat() <= density && (acceptDocs == null || acceptDocs.get(docID))) {
-          bits.set(docID);
-          //System.out.println("  acc id=" + idSource.getInt(docID) + " docID=" + docID);
+    public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+      return new ConstantScoreWeight(this) {
+        @Override
+        public Scorer scorer(LeafReaderContext context) throws IOException {
+          int maxDoc = context.reader().maxDoc();
+          FixedBitSet bits = new FixedBitSet(maxDoc);
+          Random random = new Random(seed ^ context.docBase);
+          for(int docID=0;docID<maxDoc;docID++) {
+            if (random.nextFloat() <= density) {
+              bits.set(docID);
+              //System.out.println("  acc id=" + idSource.getInt(docID) + " docID=" + docID);
+            }
+          }
+          return new ConstantScoreScorer(this, score(), new BitSetIterator(bits, bits.approximateCardinality()));
         }
-      }
-
-      return new BitDocIdSet(bits);
+      };
     }
 
     @Override
@@ -665,7 +670,7 @@ public class TestTermAutomatonQuery exte
       if (super.equals(obj) == false) {
         return false;
       }
-      RandomFilter other = (RandomFilter) obj;
+      RandomQuery other = (RandomQuery) obj;
       return seed == other.seed && density == other.density;
     }
 

Modified: lucene/dev/branches/lucene6825/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/BKD3DTreeWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/BKD3DTreeWriter.java?rev=1708778&r1=1708777&r2=1708778&view=diff
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/BKD3DTreeWriter.java (original)
+++ lucene/dev/branches/lucene6825/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/BKD3DTreeWriter.java Thu Oct 15 10:25:39 2015
@@ -18,14 +18,13 @@ package org.apache.lucene.bkdtree3d;
  */
 
 import java.io.IOException;
-import java.nio.file.DirectoryStream;
-import java.nio.file.Files;
-import java.nio.file.Path;
 import java.util.Arrays;
 import java.util.Comparator;
 
 import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.ByteArrayDataOutput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
@@ -33,8 +32,8 @@ import org.apache.lucene.util.BytesRefBu
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.InPlaceMergeSorter;
 import org.apache.lucene.util.LongBitSet;
-import org.apache.lucene.util.OfflineSorter.ByteSequencesWriter;
 import org.apache.lucene.util.OfflineSorter;
+import org.apache.lucene.util.OfflineSorter.ByteSequencesWriter;
 import org.apache.lucene.util.RamUsageEstimator;
 
 // TODO
@@ -83,10 +82,13 @@ class BKD3DTreeWriter {
   private final byte[] scratchBytes = new byte[BYTES_PER_DOC];
   private final ByteArrayDataOutput scratchBytesOutput = new ByteArrayDataOutput(scratchBytes);
 
-  private OfflineSorter.ByteSequencesWriter writer;
+  private final Directory tempDir;
+  private final String tempFileNamePrefix;
+
+  private OfflineSorter.ByteSequencesWriter offlineWriter;
   private GrowingHeapWriter heapWriter;
 
-  private Path tempInput;
+  private IndexOutput tempInput;
   private final int maxPointsInLeafNode;
   private final int maxPointsSortInHeap;
 
@@ -94,13 +96,15 @@ class BKD3DTreeWriter {
 
   private final int[] scratchDocIDs;
 
-  public BKD3DTreeWriter() throws IOException {
-    this(DEFAULT_MAX_POINTS_IN_LEAF_NODE, DEFAULT_MAX_POINTS_SORT_IN_HEAP);
+  public BKD3DTreeWriter(Directory tempDir, String tempFileNamePrefix) throws IOException {
+    this(tempDir, tempFileNamePrefix, DEFAULT_MAX_POINTS_IN_LEAF_NODE, DEFAULT_MAX_POINTS_SORT_IN_HEAP);
   }
 
   // TODO: instead of maxPointsSortInHeap, change to maxMBHeap ... the mapping is non-obvious:
-  public BKD3DTreeWriter(int maxPointsInLeafNode, int maxPointsSortInHeap) throws IOException {
+  public BKD3DTreeWriter(Directory tempDir, String tempFileNamePrefix, int maxPointsInLeafNode, int maxPointsSortInHeap) throws IOException {
     verifyParams(maxPointsInLeafNode, maxPointsSortInHeap);
+    this.tempDir = tempDir;
+    this.tempFileNamePrefix = tempFileNamePrefix;
     this.maxPointsInLeafNode = maxPointsInLeafNode;
     this.maxPointsSortInHeap = maxPointsSortInHeap;
     scratchDocIDs = new int[maxPointsInLeafNode];
@@ -128,8 +132,8 @@ class BKD3DTreeWriter {
   private void switchToOffline() throws IOException {
 
     // For each .add we just append to this input file, then in .finish we sort this input and resursively build the tree:
-    tempInput = Files.createTempFile(OfflineSorter.getDefaultTempDir(), "in", "");
-    writer = new OfflineSorter.ByteSequencesWriter(tempInput);
+    tempInput = tempDir.createTempOutput(tempFileNamePrefix, "bkd3d", IOContext.DEFAULT);
+    offlineWriter = new OfflineSorter.ByteSequencesWriter(tempInput);
     for(int i=0;i<pointCount;i++) {
       scratchBytesOutput.reset(scratchBytes);
       scratchBytesOutput.writeInt(heapWriter.xs[i]);
@@ -138,7 +142,7 @@ class BKD3DTreeWriter {
       scratchBytesOutput.writeVInt(heapWriter.docIDs[i]);
       scratchBytesOutput.writeVLong(i);
       // TODO: can/should OfflineSorter optimize the fixed-width case?
-      writer.write(scratchBytes, 0, scratchBytes.length);
+      offlineWriter.write(scratchBytes, 0, scratchBytes.length);
     }
 
     heapWriter = null;
@@ -147,7 +151,7 @@ class BKD3DTreeWriter {
   public void add(int x, int y, int z, int docID) throws IOException {
 
     if (pointCount >= maxPointsSortInHeap) {
-      if (writer == null) {
+      if (offlineWriter == null) {
         switchToOffline();
       }
       scratchBytesOutput.reset(scratchBytes);
@@ -156,7 +160,7 @@ class BKD3DTreeWriter {
       scratchBytesOutput.writeInt(z);
       scratchBytesOutput.writeVInt(docID);
       scratchBytesOutput.writeVLong(pointCount);
-      writer.write(scratchBytes, 0, scratchBytes.length);
+      offlineWriter.write(scratchBytes, 0, scratchBytes.length);
     } else {
       // Not too many points added yet, continue using heap:
       heapWriter.append(x, y, z, pointCount, docID);
@@ -167,7 +171,7 @@ class BKD3DTreeWriter {
 
   /** Changes incoming {@link ByteSequencesWriter} file to to fixed-width-per-entry file, because we need to be able to slice
    *  as we recurse in {@link #build}. */
-  private Writer convertToFixedWidth(Path in) throws IOException {
+  private Writer convertToFixedWidth(String in) throws IOException {
     BytesRefBuilder scratch = new BytesRefBuilder();
     scratch.grow(BYTES_PER_DOC);
     BytesRef bytes = scratch.get();
@@ -177,7 +181,7 @@ class BKD3DTreeWriter {
     Writer sortedWriter = null;
     boolean success = false;
     try {
-      reader = new OfflineSorter.ByteSequencesReader(in);
+      reader = new OfflineSorter.ByteSequencesReader(tempDir.openInput(in, IOContext.READONCE));
       sortedWriter = getWriter(pointCount);
       for (long i=0;i<pointCount;i++) {
         boolean result = reader.read(scratch);
@@ -328,19 +332,18 @@ class BKD3DTreeWriter {
         }
       };
 
-      Path sorted = Files.createTempFile(OfflineSorter.getDefaultTempDir(), "sorted", "");
       boolean success = false;
+      OfflineSorter sorter = new OfflineSorter(tempDir, tempFileNamePrefix, cmp);
+      String sortedFileName = sorter.sort(tempInput.getName());
       try {
-        OfflineSorter sorter = new OfflineSorter(cmp);
-        sorter.sort(tempInput, sorted);
-        Writer writer = convertToFixedWidth(sorted);
+        Writer writer = convertToFixedWidth(sortedFileName);
         success = true;
         return writer;
       } finally {
         if (success) {
-          IOUtils.rm(sorted);
+          tempDir.deleteFile(sortedFileName);
         } else {
-          IOUtils.deleteFilesIgnoringExceptions(sorted);
+          IOUtils.deleteFilesIgnoringExceptions(tempDir, sortedFileName);
         }
       }
     }
@@ -350,8 +353,8 @@ class BKD3DTreeWriter {
   public long finish(IndexOutput out) throws IOException {
     //System.out.println("\nBKDTreeWriter.finish pointCount=" + pointCount + " out=" + out + " heapWriter=" + heapWriter + " maxPointsInLeafNode=" + maxPointsInLeafNode);
 
-    if (writer != null) {
-      writer.close();
+    if (offlineWriter != null) {
+      offlineWriter.close();
     }
 
     LongBitSet bitSet = new LongBitSet(pointCount);
@@ -413,7 +416,9 @@ class BKD3DTreeWriter {
         xSortedWriter.destroy();
         ySortedWriter.destroy();
         zSortedWriter.destroy();
-        IOUtils.rm(tempInput);
+        if (tempInput != null) {
+          tempDir.deleteFile(tempInput.getName());
+        }
       } else {
         try {
           xSortedWriter.destroy();
@@ -430,7 +435,9 @@ class BKD3DTreeWriter {
         } catch (Throwable t) {
           // Suppress to keep throwing original exc
         }
-        IOUtils.deleteFilesIgnoringExceptions(tempInput);
+        if (tempInput != null) {
+          IOUtils.deleteFilesIgnoringExceptions(tempDir, tempInput.getName());
+        }
       }
     }
 
@@ -911,7 +918,7 @@ class BKD3DTreeWriter {
     if (count < maxPointsSortInHeap) {
       return new HeapWriter((int) count);
     } else {
-      return new OfflineWriter(count);
+      return new OfflineWriter(tempDir, tempFileNamePrefix, count);
     }
   }
 }

Modified: lucene/dev/branches/lucene6825/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesConsumer.java?rev=1708778&r1=1708777&r2=1708778&view=diff
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesConsumer.java (original)
+++ lucene/dev/branches/lucene6825/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesConsumer.java Thu Oct 15 10:25:39 2015
@@ -17,22 +17,23 @@ package org.apache.lucene.bkdtree3d;
  * limitations under the License.
  */
 
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.geo3d.PlanetModel;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-
 class Geo3DDocValuesConsumer extends DocValuesConsumer implements Closeable {
   final DocValuesConsumer delegate;
   final int maxPointsInLeafNode;
@@ -40,9 +41,14 @@ class Geo3DDocValuesConsumer extends Doc
   final IndexOutput out;
   final Map<Integer,Long> fieldIndexFPs = new HashMap<>();
   final SegmentWriteState state;
+  final Directory tempDir;
+  final String tempFileNamePrefix;
 
-  public Geo3DDocValuesConsumer(PlanetModel planetModel, DocValuesConsumer delegate, SegmentWriteState state, int maxPointsInLeafNode, int maxPointsSortInHeap) throws IOException {
+  public Geo3DDocValuesConsumer(Directory tempDir, String tempFileNamePrefix, PlanetModel planetModel, DocValuesConsumer delegate,
+                                SegmentWriteState state, int maxPointsInLeafNode, int maxPointsSortInHeap) throws IOException {
     BKD3DTreeWriter.verifyParams(maxPointsInLeafNode, maxPointsSortInHeap);
+    this.tempDir = tempDir;
+    this.tempFileNamePrefix = tempFileNamePrefix;
     this.delegate = delegate;
     this.maxPointsInLeafNode = maxPointsInLeafNode;
     this.maxPointsSortInHeap = maxPointsSortInHeap;
@@ -106,7 +112,7 @@ class Geo3DDocValuesConsumer extends Doc
   @Override
   public void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
     delegate.addBinaryField(field, values);
-    BKD3DTreeWriter writer = new BKD3DTreeWriter(maxPointsInLeafNode, maxPointsSortInHeap);
+    BKD3DTreeWriter writer = new BKD3DTreeWriter(tempDir, tempFileNamePrefix, maxPointsInLeafNode, maxPointsSortInHeap);
     Iterator<BytesRef> valuesIt = values.iterator();
     for (int docID=0;docID<state.segmentInfo.maxDoc();docID++) {
       assert valuesIt.hasNext();

Modified: lucene/dev/branches/lucene6825/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesFormat.java?rev=1708778&r1=1708777&r2=1708778&view=diff
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesFormat.java (original)
+++ lucene/dev/branches/lucene6825/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesFormat.java Thu Oct 15 10:25:39 2015
@@ -22,7 +22,6 @@ import org.apache.lucene.codecs.DocValue
 import org.apache.lucene.codecs.DocValuesProducer;
 import org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat;
 import org.apache.lucene.geo3d.PlanetModel;
-import org.apache.lucene.geo3d.Vector;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
 
@@ -106,7 +105,7 @@ public class Geo3DDocValuesFormat extend
 
   @Override
   public DocValuesConsumer fieldsConsumer(final SegmentWriteState state) throws IOException {
-    return new Geo3DDocValuesConsumer(planetModel, delegate.fieldsConsumer(state), state, maxPointsInLeafNode, maxPointsSortInHeap);
+    return new Geo3DDocValuesConsumer(state.directory, state.segmentInfo.name, planetModel, delegate.fieldsConsumer(state), state, maxPointsInLeafNode, maxPointsSortInHeap);
   }
 
   @Override

Modified: lucene/dev/branches/lucene6825/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/OfflineReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/OfflineReader.java?rev=1708778&r1=1708777&r2=1708778&view=diff
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/OfflineReader.java (original)
+++ lucene/dev/branches/lucene6825/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/OfflineReader.java Thu Oct 15 10:25:39 2015
@@ -17,16 +17,14 @@ package org.apache.lucene.bkdtree3d;
  * limitations under the License.
  */
 
-import java.io.BufferedInputStream;
 import java.io.IOException;
-import java.io.InputStream;
-import java.nio.file.Files;
-import java.nio.file.Path;
 
-import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
 
 final class OfflineReader implements Reader {
-  final InputStreamDataInput in;
+  final IndexInput in;
   long countLeft;
   private int x;
   private int y;
@@ -34,18 +32,9 @@ final class OfflineReader implements Rea
   private long ord;
   private int docID;
 
-  OfflineReader(Path tempFile, long start, long count) throws IOException {
-    InputStream fis = Files.newInputStream(tempFile);
-    long seekFP = start * BKD3DTreeWriter.BYTES_PER_DOC;
-    long skipped = 0;
-    while (skipped < seekFP) {
-      long inc = fis.skip(seekFP - skipped);
-      skipped += inc;
-      if (inc == 0) {
-        throw new RuntimeException("skip returned 0");
-      }
-    }
-    in = new InputStreamDataInput(new BufferedInputStream(fis));
+  OfflineReader(Directory tempDir, String tempFileName, long start, long count) throws IOException {
+    in = tempDir.openInput(tempFileName, IOContext.READONCE);
+    in.seek(start * BKD3DTreeWriter.BYTES_PER_DOC);
     this.countLeft = count;
   }
 

Modified: lucene/dev/branches/lucene6825/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/OfflineWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/OfflineWriter.java?rev=1708778&r1=1708777&r2=1708778&view=diff
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/OfflineWriter.java (original)
+++ lucene/dev/branches/lucene6825/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/OfflineWriter.java Thu Oct 15 10:25:39 2015
@@ -17,29 +17,26 @@ package org.apache.lucene.bkdtree3d;
  * limitations under the License.
  */
 
-import org.apache.lucene.store.ByteArrayDataOutput;
-import org.apache.lucene.store.OutputStreamDataOutput;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.OfflineSorter;
-
-import 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;
 
 final class OfflineWriter implements Writer {
 
-  final Path tempFile;
+  final Directory tempDir;
+  final IndexOutput out;
   final byte[] scratchBytes = new byte[BKD3DTreeWriter.BYTES_PER_DOC];
   final ByteArrayDataOutput scratchBytesOutput = new ByteArrayDataOutput(scratchBytes);      
-  final OutputStreamDataOutput out;
   final long count;
   private long countWritten;
   private boolean closed;
 
-  public OfflineWriter(long count) throws IOException {
-    tempFile = Files.createTempFile(OfflineSorter.getDefaultTempDir(), "size" + count + ".", "");
-    out = new OutputStreamDataOutput(new BufferedOutputStream(Files.newOutputStream(tempFile)));
+  public OfflineWriter(Directory tempDir, String tempFileNamePrefix, long count) throws IOException {
+    this.tempDir = tempDir;
+    out = tempDir.createTempOutput(tempFileNamePrefix, "bkd3d", IOContext.DEFAULT);
     this.count = count;
   }
     
@@ -56,7 +53,7 @@ final class OfflineWriter implements Wri
   @Override
   public Reader getReader(long start) throws IOException {
     assert closed;
-    return new OfflineReader(tempFile, start, count-start);
+    return new OfflineReader(tempDir, out.getName(), start, count-start);
   }
 
   @Override
@@ -70,11 +67,11 @@ final class OfflineWriter implements Wri
 
   @Override
   public void destroy() throws IOException {
-    IOUtils.rm(tempFile);
+    tempDir.deleteFile(out.getName());
   }
 
   @Override
   public String toString() {
-    return "OfflineWriter(count=" + count + " tempFile=" + tempFile + ")";
+    return "OfflineWriter(count=" + count + " tempFileName=" + out.getName() + ")";
   }
 }

Modified: lucene/dev/branches/lucene6825/lucene/spatial3d/src/test/org/apache/lucene/bkdtree3d/TestGeo3DPointField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/spatial3d/src/test/org/apache/lucene/bkdtree3d/TestGeo3DPointField.java?rev=1708778&r1=1708777&r2=1708778&view=diff
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/spatial3d/src/test/org/apache/lucene/bkdtree3d/TestGeo3DPointField.java (original)
+++ lucene/dev/branches/lucene6825/lucene/spatial3d/src/test/org/apache/lucene/bkdtree3d/TestGeo3DPointField.java Thu Oct 15 10:25:39 2015
@@ -17,6 +17,16 @@ package org.apache.lucene.bkdtree3d;
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
+
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.lucene53.Lucene53Codec;
@@ -50,6 +60,7 @@ 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.MockDirectoryWrapper;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
@@ -58,16 +69,6 @@ import org.junit.BeforeClass;
 
 import com.carrotsearch.randomizedtesting.generators.RandomInts;
 
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicBoolean;
-
 import static org.apache.lucene.bkdtree3d.Geo3DDocValuesFormat.decodeValueCenter;
 import static org.apache.lucene.bkdtree3d.Geo3DDocValuesFormat.decodeValueMax;
 import static org.apache.lucene.bkdtree3d.Geo3DDocValuesFormat.decodeValueMin;
@@ -87,7 +88,7 @@ public class TestGeo3DPointField extends
   }
 
   public void testBasic() throws Exception {
-    Directory dir = newDirectory();
+    Directory dir = getDirectory();
     int maxPointsInLeaf = TestUtil.nextInt(random(), 16, 2048);
     int maxPointsSortInHeap = TestUtil.nextInt(random(), maxPointsInLeaf, 1024*1024);
     IndexWriterConfig iwc = newIndexWriterConfig();
@@ -108,7 +109,7 @@ public class TestGeo3DPointField extends
   }
 
   public void testPlanetModelChanged() throws Exception {
-    Directory dir = newDirectory();
+    Directory dir = getDirectory();
     int maxPointsInLeaf = TestUtil.nextInt(random(), 16, 2048);
     int maxPointsSortInHeap = TestUtil.nextInt(random(), maxPointsInLeaf, 1024*1024);
     IndexWriterConfig iwc = newIndexWriterConfig();
@@ -137,10 +138,10 @@ public class TestGeo3DPointField extends
   }
 
   public void testBKDBasic() throws Exception {
-    Directory dir = newDirectory();
+    Directory dir = getDirectory();
     IndexOutput out = dir.createOutput("bkd", IOContext.DEFAULT);
 
-    BKD3DTreeWriter w = new BKD3DTreeWriter();
+    BKD3DTreeWriter w = new BKD3DTreeWriter(dir, "bkd3d");
 
     w.add(0, 0, 0, 0);
     w.add(1, 1, 1, 1);
@@ -245,7 +246,7 @@ public class TestGeo3DPointField extends
   public void testBKDRandom() throws Exception {
     List<Point> points = new ArrayList<>();
     int numPoints = atLeast(10000);
-    Directory dir = newDirectory();
+    Directory dir = getDirectory();
     IndexOutput out = dir.createOutput("bkd", IOContext.DEFAULT);
     int maxPointsInLeaf = TestUtil.nextInt(random(), 16, 2048); 
 
@@ -254,7 +255,7 @@ public class TestGeo3DPointField extends
     PlanetModel planetModel = getPlanetModel();
     final double planetMax = planetModel.getMaximumMagnitude();
     
-    BKD3DTreeWriter w = new BKD3DTreeWriter(maxPointsInLeaf, maxPointsSortInHeap);
+    BKD3DTreeWriter w = new BKD3DTreeWriter(dir, "bkd3d", maxPointsInLeaf, maxPointsSortInHeap);
     for(int docID=0;docID<numPoints;docID++) {
       Point point;
       if (docID > 0 && random().nextInt(30) == 17) {
@@ -924,7 +925,7 @@ public class TestGeo3DPointField extends
     if (lats.length > 100000) {
       dir = newFSDirectory(createTempDir("TestBKDTree"));
     } else {
-      dir = newDirectory();
+      dir = getDirectory();
     }
     Set<Integer> deleted = new HashSet<>();
     // RandomIndexWriter is too slow here:
@@ -1059,4 +1060,12 @@ public class TestGeo3DPointField extends
     }
     IOUtils.close(r, dir);
   }
+
+  private static Directory getDirectory() {     
+    Directory dir = newDirectory();
+    if (dir instanceof MockDirectoryWrapper) {
+      ((MockDirectoryWrapper) dir).setEnableVirusScanner(false);
+    }
+    return dir;
+  }
 }

Modified: lucene/dev/branches/lucene6825/lucene/suggest/src/java/org/apache/lucene/search/suggest/Lookup.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/suggest/src/java/org/apache/lucene/search/suggest/Lookup.java?rev=1708778&r1=1708777&r2=1708778&view=diff
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/suggest/src/java/org/apache/lucene/search/suggest/Lookup.java (original)
+++ lucene/dev/branches/lucene6825/lucene/suggest/src/java/org/apache/lucene/search/suggest/Lookup.java Thu Oct 15 10:25:39 2015
@@ -24,6 +24,7 @@ import java.util.Comparator;
 import java.util.List;
 import java.util.Set;
 
+import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.spell.Dictionary;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
@@ -252,6 +253,22 @@ public abstract class Lookup implements
   public abstract List<LookupResult> lookup(CharSequence key, Set<BytesRef> contexts, boolean onlyMorePopular, int num) throws IOException;
 
   /**
+   * Look up a key and return possible completion for this key.
+   * This needs to be overridden by all implementing classes as the default implementation just returns null
+   *
+   * @param key the lookup key
+   * @param contextFilerQuery A query for further filtering the result of the key lookup
+   * @param num maximum number of results to return
+   * @param allTermsRequired true is all terms are required
+   * @param doHighlight set to true if key should be highlighted
+   * @return a list of suggestions/completions. The default implementation returns null, meaning each @Lookup implementation should override this and provide their own implementation
+   * @throws IOException when IO exception occurs
+   */
+  public List<LookupResult> lookup(CharSequence key, BooleanQuery contextFilerQuery, int num, boolean allTermsRequired, boolean doHighlight) throws IOException{
+    return null;
+  }
+
+  /**
    * Persist the constructed lookup data to a directory. Optional operation.
    * @param output {@link DataOutput} to write the data to.
    * @return true if successful, false if unsuccessful or not supported.

Modified: lucene/dev/branches/lucene6825/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedInputIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedInputIterator.java?rev=1708778&r1=1708777&r2=1708778&view=diff
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedInputIterator.java (original)
+++ lucene/dev/branches/lucene6825/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedInputIterator.java Thu Oct 15 10:25:39 2015
@@ -18,14 +18,15 @@ package org.apache.lucene.search.suggest
  */
 
 import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Path;
 import java.util.Comparator;
 import java.util.HashSet;
 import java.util.Set;
 
 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.IndexOutput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
@@ -41,12 +42,14 @@ import org.apache.lucene.util.OfflineSor
 public class SortedInputIterator implements InputIterator {
   
   private final InputIterator source;
-  private Path tempInput;
-  private Path tempSorted;
+  private IndexOutput tempInput;
+  private String tempSortedFileName;
   private final ByteSequencesReader reader;
   private final Comparator<BytesRef> comparator;
   private final boolean hasPayloads;
   private final boolean hasContexts;
+  private final Directory tempDir;
+  private final String tempFileNamePrefix;
   private boolean done = false;
   
   private long weight;
@@ -58,19 +61,21 @@ public class SortedInputIterator impleme
    * Creates a new sorted wrapper, using {@link
    * BytesRef#getUTF8SortedAsUnicodeComparator} for
    * sorting. */
-  public SortedInputIterator(InputIterator source) throws IOException {
-    this(source, BytesRef.getUTF8SortedAsUnicodeComparator());
+  public SortedInputIterator(Directory tempDir, String tempFileNamePrefix, InputIterator source) throws IOException {
+    this(tempDir, tempFileNamePrefix, source, BytesRef.getUTF8SortedAsUnicodeComparator());
   }
 
   /**
    * Creates a new sorted wrapper, sorting by BytesRef
    * (ascending) then cost (ascending).
    */
-  public SortedInputIterator(InputIterator source, Comparator<BytesRef> comparator) throws IOException {
+  public SortedInputIterator(Directory tempDir, String tempFileNamePrefix, InputIterator source, Comparator<BytesRef> comparator) throws IOException {
     this.hasPayloads = source.hasPayloads();
     this.hasContexts = source.hasContexts();
     this.source = source;
     this.comparator = comparator;
+    this.tempDir = tempDir;
+    this.tempFileNamePrefix = tempFileNamePrefix;
     this.reader = sort();
   }
   
@@ -83,7 +88,7 @@ public class SortedInputIterator impleme
     try {
       ByteArrayDataInput input = new ByteArrayDataInput();
       if (reader.read(scratch)) {
-      final BytesRef bytes = scratch.get();
+        final BytesRef bytes = scratch.get();
         weight = decode(bytes, input);
         if (hasPayloads) {
           payload = decodePayload(bytes, input);
@@ -168,10 +173,9 @@ public class SortedInputIterator impleme
   };
   
   private ByteSequencesReader sort() throws IOException {
-    String prefix = getClass().getSimpleName();
-    Path directory = OfflineSorter.getDefaultTempDir();
-    tempInput = Files.createTempFile(directory, prefix, ".input");
-    tempSorted = Files.createTempFile(directory, prefix, ".sorted");
+
+    OfflineSorter sorter = new OfflineSorter(tempDir, tempFileNamePrefix, tieBreakByCostComparator);
+    tempInput = tempDir.createTempOutput(tempFileNamePrefix, "input", IOContext.DEFAULT);
     
     final OfflineSorter.ByteSequencesWriter writer = new OfflineSorter.ByteSequencesWriter(tempInput);
     boolean success = false;
@@ -184,8 +188,8 @@ public class SortedInputIterator impleme
         encode(writer, output, buffer, spare, source.payload(), source.contexts(), source.weight());
       }
       writer.close();
-      new OfflineSorter(tieBreakByCostComparator).sort(tempInput, tempSorted);
-      ByteSequencesReader reader = new OfflineSorter.ByteSequencesReader(tempSorted);
+      tempSortedFileName = sorter.sort(tempInput.getName());
+      ByteSequencesReader reader = new OfflineSorter.ByteSequencesReader(tempDir.openInput(tempSortedFileName, IOContext.READONCE));
       success = true;
       return reader;
       
@@ -203,16 +207,12 @@ public class SortedInputIterator impleme
   }
   
   private void close() throws IOException {
-    boolean success = false;
     try {
       IOUtils.close(reader);
-      success = true;
     } finally {
-      if (success) {
-        IOUtils.deleteFilesIfExist(tempInput, tempSorted);
-      } else {
-        IOUtils.deleteFilesIgnoringExceptions(tempInput, tempSorted);
-      }
+      IOUtils.deleteFilesIgnoringExceptions(tempDir,
+                                            tempInput == null ? null : tempInput.getName(),
+                                            tempSortedFileName);
     }
   }
   

Modified: lucene/dev/branches/lucene6825/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java?rev=1708778&r1=1708777&r2=1708778&view=diff
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java (original)
+++ lucene/dev/branches/lucene6825/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java Thu Oct 15 10:25:39 2015
@@ -17,11 +17,7 @@ package org.apache.lucene.search.suggest
  * limitations under the License.
  */
 
-import static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZED_STATES;
-
 import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -39,6 +35,9 @@ import org.apache.lucene.store.ByteArray
 import org.apache.lucene.store.ByteArrayDataOutput;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.ArrayUtil;
@@ -64,6 +63,8 @@ import org.apache.lucene.util.fst.Util;
 import org.apache.lucene.util.fst.Util.Result;
 import org.apache.lucene.util.fst.Util.TopResults;
 
+import static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZED_STATES;
+
 /**
  * Suggester that first analyzes the surface form, adds the
  * analyzed form to a weighted FST, and then does the same
@@ -150,14 +151,14 @@ public class AnalyzingSuggester extends
   private final boolean preserveSep;
 
   /** Include this flag in the options parameter to {@link
-   *  #AnalyzingSuggester(Analyzer,Analyzer,int,int,int,boolean)} to always
+   *  #AnalyzingSuggester(Directory,String,Analyzer,Analyzer,int,int,int,boolean)} to always
    *  return the exact match first, regardless of score.  This
    *  has no performance impact but could result in
    *  low-quality suggestions. */
   public static final int EXACT_FIRST = 1;
 
   /** Include this flag in the options parameter to {@link
-   *  #AnalyzingSuggester(Analyzer,Analyzer,int,int,int,boolean)} to preserve
+   *  #AnalyzingSuggester(Directory,String,Analyzer,Analyzer,int,int,int,boolean)} to preserve
    *  token separators when matching. */
   public static final int PRESERVE_SEP = 2;
 
@@ -179,6 +180,9 @@ public class AnalyzingSuggester extends
    *  SynonymFilter). */
   private final int maxGraphExpansions;
 
+  private final Directory tempDir;
+  private final String tempFileNamePrefix;
+
   /** Highest number of analyzed paths we saw for any single
    *  input surface form.  For analyzers that never create
    *  graphs this will always be 1. */
@@ -195,21 +199,21 @@ public class AnalyzingSuggester extends
   private long count = 0;
 
   /**
-   * Calls {@link #AnalyzingSuggester(Analyzer,Analyzer,int,int,int,boolean)
+   * Calls {@link #AnalyzingSuggester(Directory,String,Analyzer,Analyzer,int,int,int,boolean)
    * AnalyzingSuggester(analyzer, analyzer, EXACT_FIRST |
    * PRESERVE_SEP, 256, -1, true)}
    */
-  public AnalyzingSuggester(Analyzer analyzer) {
-    this(analyzer, analyzer, EXACT_FIRST | PRESERVE_SEP, 256, -1, true);
+  public AnalyzingSuggester(Directory tempDir, String tempFileNamePrefix, Analyzer analyzer) {
+    this(tempDir, tempFileNamePrefix, analyzer, analyzer, EXACT_FIRST | PRESERVE_SEP, 256, -1, true);
   }
 
   /**
-   * Calls {@link #AnalyzingSuggester(Analyzer,Analyzer,int,int,int,boolean)
+   * Calls {@link #AnalyzingSuggester(Directory,String,Analyzer,Analyzer,int,int,int,boolean)
    * AnalyzingSuggester(indexAnalyzer, queryAnalyzer, EXACT_FIRST |
    * PRESERVE_SEP, 256, -1, true)}
    */
-  public AnalyzingSuggester(Analyzer indexAnalyzer, Analyzer queryAnalyzer) {
-    this(indexAnalyzer, queryAnalyzer, EXACT_FIRST | PRESERVE_SEP, 256, -1, true);
+  public AnalyzingSuggester(Directory tempDir, String tempFileNamePrefix, Analyzer indexAnalyzer, Analyzer queryAnalyzer) {
+    this(tempDir, tempFileNamePrefix, indexAnalyzer, queryAnalyzer, EXACT_FIRST | PRESERVE_SEP, 256, -1, true);
   }
 
   /**
@@ -230,7 +234,7 @@ public class AnalyzingSuggester extends
    * @param preservePositionIncrements Whether position holes
    *   should appear in the automata
    */
-  public AnalyzingSuggester(Analyzer indexAnalyzer, Analyzer queryAnalyzer, int options, int maxSurfaceFormsPerAnalyzedForm, int maxGraphExpansions,
+  public AnalyzingSuggester(Directory tempDir, String tempFileNamePrefix, Analyzer indexAnalyzer, Analyzer queryAnalyzer, int options, int maxSurfaceFormsPerAnalyzedForm, int maxGraphExpansions,
       boolean preservePositionIncrements) {
     this.indexAnalyzer = indexAnalyzer;
     this.queryAnalyzer = queryAnalyzer;
@@ -254,6 +258,8 @@ public class AnalyzingSuggester extends
     }
     this.maxGraphExpansions = maxGraphExpansions;
     this.preservePositionIncrements = preservePositionIncrements;
+    this.tempDir = tempDir;
+    this.tempFileNamePrefix = tempFileNamePrefix;
   }
 
   /** Returns byte size of the underlying FST. */
@@ -396,20 +402,21 @@ public class AnalyzingSuggester extends
     if (iterator.hasContexts()) {
       throw new IllegalArgumentException("this suggester doesn't support contexts");
     }
-    String prefix = getClass().getSimpleName();
-    Path directory = OfflineSorter.getDefaultTempDir();
-    Path tempInput = Files.createTempFile(directory, prefix, ".input");
-    Path tempSorted = Files.createTempFile(directory, prefix, ".sorted");
 
     hasPayloads = iterator.hasPayloads();
 
+    OfflineSorter sorter = new OfflineSorter(tempDir, tempFileNamePrefix, new AnalyzingComparator(hasPayloads));
+
+    IndexOutput tempInput = tempDir.createTempOutput(tempFileNamePrefix, "input", IOContext.DEFAULT);
+
     OfflineSorter.ByteSequencesWriter writer = new OfflineSorter.ByteSequencesWriter(tempInput);
     OfflineSorter.ByteSequencesReader reader = null;
     BytesRefBuilder scratch = new BytesRefBuilder();
 
     TokenStreamToAutomaton ts2a = getTokenStreamToAutomaton();
 
-    boolean success = false;
+    String tempSortedFileName = null;
+
     count = 0;
     byte buffer[] = new byte[8];
     try {
@@ -477,12 +484,12 @@ public class AnalyzingSuggester extends
       writer.close();
 
       // Sort all input/output pairs (required by FST.Builder):
-      new OfflineSorter(new AnalyzingComparator(hasPayloads)).sort(tempInput, tempSorted);
+      tempSortedFileName = sorter.sort(tempInput.getName());
 
       // Free disk space:
-      Files.delete(tempInput);
+      tempDir.deleteFile(tempInput.getName());
 
-      reader = new OfflineSorter.ByteSequencesReader(tempSorted);
+      reader = new OfflineSorter.ByteSequencesReader(tempDir.openInput(tempSortedFileName, IOContext.READONCE));
      
       PairOutputs<Long,BytesRef> outputs = new PairOutputs<>(PositiveIntOutputs.getSingleton(), ByteSequenceOutputs.getSingleton());
       Builder<Pair<Long,BytesRef>> builder = new Builder<>(FST.INPUT_TYPE.BYTE1, outputs);
@@ -570,16 +577,9 @@ public class AnalyzingSuggester extends
       fst = builder.finish();
 
       //Util.dotToFile(fst, "/tmp/suggest.dot");
-      
-      success = true;
     } finally {
       IOUtils.closeWhileHandlingException(reader, writer);
-      
-      if (success) {
-        IOUtils.deleteFilesIfExist(tempInput, tempSorted);
-      } else {
-        IOUtils.deleteFilesIgnoringExceptions(tempInput, tempSorted);
-      }
+      IOUtils.deleteFilesIgnoringExceptions(tempDir, tempInput.getName(), tempSortedFileName);
     }
   }
 

Modified: lucene/dev/branches/lucene6825/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6825/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java?rev=1708778&r1=1708777&r2=1708778&view=diff
==============================================================================
--- lucene/dev/branches/lucene6825/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java (original)
+++ lucene/dev/branches/lucene6825/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java Thu Oct 15 10:25:39 2015
@@ -24,6 +24,7 @@ import org.apache.lucene.analysis.Analyz
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.TokenStreamToAutomaton;
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute; // javadocs
+import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.UnicodeUtil;
@@ -113,8 +114,8 @@ public final class FuzzySuggester extend
    * 
    * @param analyzer the analyzer used for this suggester
    */
-  public FuzzySuggester(Analyzer analyzer) {
-    this(analyzer, analyzer);
+  public FuzzySuggester(Directory tempDir, String tempFileNamePrefix, Analyzer analyzer) {
+    this(tempDir, tempFileNamePrefix, analyzer, analyzer);
   }
   
   /**
@@ -125,8 +126,8 @@ public final class FuzzySuggester extend
    * @param queryAnalyzer
    *           Analyzer that will be used for analyzing query text during lookup
    */
-  public FuzzySuggester(Analyzer indexAnalyzer, Analyzer queryAnalyzer) {
-    this(indexAnalyzer, queryAnalyzer, EXACT_FIRST | PRESERVE_SEP, 256, -1, true, DEFAULT_MAX_EDITS, DEFAULT_TRANSPOSITIONS,
+  public FuzzySuggester(Directory tempDir, String tempFileNamePrefix, Analyzer indexAnalyzer, Analyzer queryAnalyzer) {
+    this(tempDir, tempFileNamePrefix, indexAnalyzer, queryAnalyzer, EXACT_FIRST | PRESERVE_SEP, 256, -1, true, DEFAULT_MAX_EDITS, DEFAULT_TRANSPOSITIONS,
          DEFAULT_NON_FUZZY_PREFIX, DEFAULT_MIN_FUZZY_LENGTH, DEFAULT_UNICODE_AWARE);
   }
 
@@ -154,11 +155,11 @@ public final class FuzzySuggester extend
    * @param minFuzzyLength minimum length of lookup key before any edits are allowed (see default {@link #DEFAULT_MIN_FUZZY_LENGTH})
    * @param unicodeAware operate Unicode code points instead of bytes.
    */
-  public FuzzySuggester(Analyzer indexAnalyzer, Analyzer queryAnalyzer,
+  public FuzzySuggester(Directory tempDir, String tempFileNamePrefix, Analyzer indexAnalyzer, Analyzer queryAnalyzer,
                         int options, int maxSurfaceFormsPerAnalyzedForm, int maxGraphExpansions,
                         boolean preservePositionIncrements, int maxEdits, boolean transpositions,
                         int nonFuzzyPrefix, int minFuzzyLength, boolean unicodeAware) {
-    super(indexAnalyzer, queryAnalyzer, options, maxSurfaceFormsPerAnalyzedForm, maxGraphExpansions, preservePositionIncrements);
+    super(tempDir, tempFileNamePrefix, indexAnalyzer, queryAnalyzer, options, maxSurfaceFormsPerAnalyzedForm, maxGraphExpansions, preservePositionIncrements);
     if (maxEdits < 0 || maxEdits > LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE) {
       throw new IllegalArgumentException("maxEdits must be between 0 and " + LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE);
     }

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=1708778&r1=1708777&r2=1708778&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 Thu Oct 15 10:25:39 2015
@@ -19,10 +19,10 @@ package org.apache.lucene.search.suggest
 
 import java.io.Closeable;
 import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Path;
 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.BytesRefIterator;
 import org.apache.lucene.util.IOUtils;
@@ -34,48 +34,49 @@ import org.apache.lucene.util.OfflineSor
  * @lucene.internal
  */
 public class ExternalRefSorter implements BytesRefSorter, Closeable {
-  private final OfflineSorter sort;
+  private final OfflineSorter sorter;
   private OfflineSorter.ByteSequencesWriter writer;
-  private Path input;
-  private Path sorted;
+  private IndexOutput input;
+  private String sortedFileName;
   
   /**
    * Will buffer all sequences to a temporary file and then sort (all on-disk).
    */
-  public ExternalRefSorter(OfflineSorter sort) throws IOException {
-    this.sort = sort;
-    this.input = Files.createTempFile(OfflineSorter.getDefaultTempDir(), "RefSorter-", ".raw");
-    this.writer = new OfflineSorter.ByteSequencesWriter(input);
+  public ExternalRefSorter(OfflineSorter sorter) throws IOException {
+    this.sorter = sorter;
+    this.input = sorter.getDirectory().createTempOutput(sorter.getTempFileNamePrefix(), "RefSorterRaw", IOContext.DEFAULT);
+    this.writer = new OfflineSorter.ByteSequencesWriter(this.input);
   }
   
   @Override
   public void add(BytesRef utf8) throws IOException {
-    if (writer == null) throw new IllegalStateException();
+    if (writer == null) {
+      throw new IllegalStateException();
+    }
     writer.write(utf8);
   }
   
   @Override
   public BytesRefIterator iterator() throws IOException {
-    if (sorted == null) {
+    if (sortedFileName == null) {
       closeWriter();
       
-      sorted = Files.createTempFile(OfflineSorter.getDefaultTempDir(), "RefSorter-", ".sorted");
       boolean success = false;
       try {
-        sort.sort(input, sorted);
+        sortedFileName = sorter.sort(input.getName());
         success = true;
       } finally {
         if (success) {
-          Files.delete(input);
+          sorter.getDirectory().deleteFile(input.getName());
         } else {
-          IOUtils.deleteFilesIgnoringExceptions(input);
+          IOUtils.deleteFilesIgnoringExceptions(sorter.getDirectory(), input.getName());
         }
       }
       
       input = null;
     }
     
-    return new ByteSequenceIterator(new OfflineSorter.ByteSequencesReader(sorted));
+    return new ByteSequenceIterator(new OfflineSorter.ByteSequencesReader(sorter.getDirectory().openInput(sortedFileName, IOContext.READONCE)));
   }
   
   private void closeWriter() throws IOException {
@@ -90,16 +91,12 @@ public class ExternalRefSorter implement
    */
   @Override
   public void close() throws IOException {
-    boolean success = false;
     try {
       closeWriter();
-      success = true;
     } finally {
-      if (success) {
-        IOUtils.deleteFilesIfExist(input, sorted);
-      } else {
-        IOUtils.deleteFilesIgnoringExceptions(input, sorted);
-      }
+      IOUtils.deleteFilesIgnoringExceptions(sorter.getDirectory(),
+                                            input == null ? null : input.getName(),
+                                            sortedFileName);
     }
   }
   
@@ -142,6 +139,6 @@ public class ExternalRefSorter implement
 
   @Override
   public Comparator<BytesRef> getComparator() {
-    return sort.getComparator();
+    return sorter.getComparator();
   }
 }