lucene-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From mikemcc...@apache.org
Subject svn commit: r1709414 [3/5] - in /lucene/dev/branches/lucene6835: ./ 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...
Date Mon, 19 Oct 2015 13:38:44 GMT
Modified: lucene/dev/branches/lucene6835/lucene/sandbox/src/test/org/apache/lucene/rangetree/TestRangeTree.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/sandbox/src/test/org/apache/lucene/rangetree/TestRangeTree.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/sandbox/src/test/org/apache/lucene/rangetree/TestRangeTree.java (original)
+++ lucene/dev/branches/lucene6835/lucene/sandbox/src/test/org/apache/lucene/rangetree/TestRangeTree.java Mon Oct 19 13:38:42 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/lucene6835/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/BKD3DTreeWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/BKD3DTreeWriter.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/BKD3DTreeWriter.java (original)
+++ lucene/dev/branches/lucene6835/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/BKD3DTreeWriter.java Mon Oct 19 13:38:42 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/lucene6835/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesConsumer.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesConsumer.java (original)
+++ lucene/dev/branches/lucene6835/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesConsumer.java Mon Oct 19 13:38:42 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/lucene6835/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesFormat.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesFormat.java (original)
+++ lucene/dev/branches/lucene6835/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesFormat.java Mon Oct 19 13:38:42 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/lucene6835/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/OfflineReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/OfflineReader.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/OfflineReader.java (original)
+++ lucene/dev/branches/lucene6835/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/OfflineReader.java Mon Oct 19 13:38:42 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/lucene6835/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/OfflineWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/OfflineWriter.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/OfflineWriter.java (original)
+++ lucene/dev/branches/lucene6835/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/OfflineWriter.java Mon Oct 19 13:38:42 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/lucene6835/lucene/spatial3d/src/test/org/apache/lucene/bkdtree3d/TestGeo3DPointField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/spatial3d/src/test/org/apache/lucene/bkdtree3d/TestGeo3DPointField.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/spatial3d/src/test/org/apache/lucene/bkdtree3d/TestGeo3DPointField.java (original)
+++ lucene/dev/branches/lucene6835/lucene/spatial3d/src/test/org/apache/lucene/bkdtree3d/TestGeo3DPointField.java Mon Oct 19 13:38:42 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/lucene6835/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedInputIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedInputIterator.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedInputIterator.java (original)
+++ lucene/dev/branches/lucene6835/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedInputIterator.java Mon Oct 19 13:38:42 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/lucene6835/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java (original)
+++ lucene/dev/branches/lucene6835/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java Mon Oct 19 13:38:42 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/lucene6835/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java (original)
+++ lucene/dev/branches/lucene6835/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java Mon Oct 19 13:38:42 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/lucene6835/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java (original)
+++ lucene/dev/branches/lucene6835/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java Mon Oct 19 13:38:42 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();
   }
 }

Modified: lucene/dev/branches/lucene6835/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java (original)
+++ lucene/dev/branches/lucene6835/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java Mon Oct 19 13:38:42 2015
@@ -18,8 +18,6 @@ package org.apache.lucene.search.suggest
  */
 
 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;
@@ -34,6 +32,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;
@@ -42,7 +43,6 @@ import org.apache.lucene.util.BytesRefBu
 import org.apache.lucene.util.CharsRefBuilder;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.OfflineSorter;
-import org.apache.lucene.util.OfflineSorter.SortInfo;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.fst.FST;
 import org.apache.lucene.util.fst.NoOutputs;
@@ -76,7 +76,7 @@ public class FSTCompletionLookup extends
    * An invalid bucket count if we're creating an object
    * of this class from an existing FST.
    * 
-   * @see #FSTCompletionLookup(FSTCompletion, boolean)
+   * @see #FSTCompletionLookup(Directory, String, FSTCompletion, boolean)
    */
   private static int INVALID_BUCKETS_COUNT = -1;
   
@@ -89,6 +89,9 @@ public class FSTCompletionLookup extends
    */
   private final static int sharedTailLength = 5;
 
+  private final Directory tempDir;
+  private final String tempFileNamePrefix;
+
   private int buckets;
   private boolean exactMatchFirst;
 
@@ -106,13 +109,20 @@ public class FSTCompletionLookup extends
   private long count = 0;
 
   /**
+   * This constructor should only be used to read a previously saved suggester.
+   */
+  public FSTCompletionLookup() {
+    this(null, null);
+  }
+
+  /**
    * This constructor prepares for creating a suggested FST using the
    * {@link #build(InputIterator)} method. The number of weight
    * discretization buckets is set to {@link FSTCompletion#DEFAULT_BUCKETS} and
    * exact matches are promoted to the top of the suggestions list.
    */
-  public FSTCompletionLookup() {
-    this(FSTCompletion.DEFAULT_BUCKETS, true);
+  public FSTCompletionLookup(Directory tempDir, String tempFileNamePrefix) {
+    this(tempDir, tempFileNamePrefix, FSTCompletion.DEFAULT_BUCKETS, true);
   }
 
   /**
@@ -128,9 +138,11 @@ public class FSTCompletionLookup extends
    *          suggestions list. Otherwise they appear in the order of
    *          discretized weight and alphabetical within the bucket.
    */
-  public FSTCompletionLookup(int buckets, boolean exactMatchFirst) {
+  public FSTCompletionLookup(Directory tempDir, String tempFileNamePrefix, int buckets, boolean exactMatchFirst) {
     this.buckets = buckets;
     this.exactMatchFirst = exactMatchFirst;
+    this.tempDir = tempDir;
+    this.tempFileNamePrefix = tempFileNamePrefix;
   }
 
   /**
@@ -143,8 +155,8 @@ public class FSTCompletionLookup extends
    *          suggestions list. Otherwise they appear in the order of
    *          discretized weight and alphabetical within the bucket.
    */
-  public FSTCompletionLookup(FSTCompletion completion, boolean exactMatchFirst) {
-    this(INVALID_BUCKETS_COUNT, exactMatchFirst);
+  public FSTCompletionLookup(Directory tempDir, String tempFileNamePrefix, FSTCompletion completion, boolean exactMatchFirst) {
+    this(tempDir, tempFileNamePrefix, INVALID_BUCKETS_COUNT, exactMatchFirst);
     this.normalCompletion = new FSTCompletion(
         completion.getFST(), false, exactMatchFirst);
     this.higherWeightsCompletion =  new FSTCompletion(
@@ -159,23 +171,23 @@ public class FSTCompletionLookup extends
     if (iterator.hasContexts()) {
       throw new IllegalArgumentException("this suggester doesn't support contexts");
     }
-    Path tempInput = Files.createTempFile(
-        OfflineSorter.getDefaultTempDir(), FSTCompletionLookup.class.getSimpleName(), ".input");
-    Path tempSorted = Files.createTempFile(
-        OfflineSorter.getDefaultTempDir(), FSTCompletionLookup.class.getSimpleName(), ".sorted");
+
+    OfflineSorter sorter = new OfflineSorter(tempDir, tempFileNamePrefix);
+    ExternalRefSorter externalSorter = new ExternalRefSorter(sorter);
+    IndexOutput tempInput = tempDir.createTempOutput(tempFileNamePrefix, "input", IOContext.DEFAULT);
+    String tempSortedFileName = null;
 
     OfflineSorter.ByteSequencesWriter writer = new OfflineSorter.ByteSequencesWriter(tempInput);
     OfflineSorter.ByteSequencesReader reader = null;
-    ExternalRefSorter sorter = null;
 
     // Push floats up front before sequences to sort them. For now, assume they are non-negative.
     // If negative floats are allowed some trickery needs to be done to find their byte order.
-    boolean success = false;
     count = 0;
     try {
       byte [] buffer = new byte [0];
       ByteArrayDataOutput output = new ByteArrayDataOutput(buffer);
       BytesRef spare;
+      int inputLineCount = 0;
       while ((spare = iterator.next()) != null) {
         if (spare.length + 4 >= buffer.length) {
           buffer = ArrayUtil.grow(buffer, spare.length + 4);
@@ -185,18 +197,19 @@ public class FSTCompletionLookup extends
         output.writeInt(encodeWeight(iterator.weight()));
         output.writeBytes(spare.bytes, spare.offset, spare.length);
         writer.write(buffer, 0, output.getPosition());
+        inputLineCount++;
       }
       writer.close();
 
       // We don't know the distribution of scores and we need to bucket them, so we'll sort
       // and divide into equal buckets.
-      SortInfo info = new OfflineSorter().sort(tempInput, tempSorted);
-      Files.delete(tempInput);
+      tempSortedFileName = sorter.sort(tempInput.getName());
+      tempDir.deleteFile(tempInput.getName());
+
       FSTCompletionBuilder builder = new FSTCompletionBuilder(
-          buckets, sorter = new ExternalRefSorter(new OfflineSorter()), sharedTailLength);
+          buckets, externalSorter, sharedTailLength);
 
-      final int inputLines = info.lines;
-      reader = new OfflineSorter.ByteSequencesReader(tempSorted);
+      reader = new OfflineSorter.ByteSequencesReader(tempDir.openInput(tempSortedFileName, IOContext.READONCE));
       long line = 0;
       int previousBucket = 0;
       int previousScore = 0;
@@ -211,7 +224,7 @@ public class FSTCompletionLookup extends
         if (line > 0 && currentScore == previousScore) {
           bucket = previousBucket;
         } else {
-          bucket = (int) (line * buckets / inputLines);
+          bucket = (int) (line * buckets / inputLineCount);
         }
         previousScore = currentScore;
         previousBucket = bucket;
@@ -231,15 +244,9 @@ public class FSTCompletionLookup extends
       this.normalCompletion = new FSTCompletion(
           higherWeightsCompletion.getFST(), false, exactMatchFirst);
       
-      success = true;
     } finally {
-      IOUtils.closeWhileHandlingException(reader, writer, sorter);
-
-      if (success) {
-        Files.delete(tempSorted);
-      } else {
-        IOUtils.deleteFilesIgnoringExceptions(tempInput, tempSorted);
-      }
+      IOUtils.closeWhileHandlingException(reader, writer, externalSorter);
+      IOUtils.deleteFilesIgnoringExceptions(tempDir, tempInput.getName(), tempSortedFileName);
     }
   }
   
@@ -285,8 +292,9 @@ public class FSTCompletionLookup extends
   @Override
   public synchronized boolean store(DataOutput output) throws IOException {
     output.writeVLong(count);
-    if (this.normalCompletion == null || normalCompletion.getFST() == null) 
+    if (this.normalCompletion == null || normalCompletion.getFST() == null) {
       return false;
+    }
     normalCompletion.getFST().save(output);
     return true;
   }

Modified: lucene/dev/branches/lucene6835/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java (original)
+++ lucene/dev/branches/lucene6835/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java Mon Oct 19 13:38:42 2015
@@ -32,6 +32,7 @@ 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.util.Accountable;
 import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.ArrayUtil;
@@ -78,11 +79,14 @@ public class WFSTCompletionLookup extend
   /** Number of entries the lookup was built with */
   private long count = 0;
 
+  private final Directory tempDir;
+  private final String tempFileNamePrefix;
+
   /**
-   * Calls {@link #WFSTCompletionLookup(boolean) WFSTCompletionLookup(true)}
+   * Calls {@link #WFSTCompletionLookup(Directory,String,boolean) WFSTCompletionLookup(null,null,true)}
    */
-  public WFSTCompletionLookup() {
-    this(true);
+  public WFSTCompletionLookup(Directory tempDir, String tempFileNamePrefix) {
+    this(tempDir, tempFileNamePrefix, true);
   }
   
   /**
@@ -93,8 +97,10 @@ public class WFSTCompletionLookup extend
    *        of score. This has no performance impact, but could result
    *        in low-quality suggestions.
    */
-  public WFSTCompletionLookup(boolean exactFirst) {
+  public WFSTCompletionLookup(Directory tempDir, String tempFileNamePrefix, boolean exactFirst) {
     this.exactFirst = exactFirst;
+    this.tempDir = tempDir;
+    this.tempFileNamePrefix = tempFileNamePrefix;
   }
   
   @Override
@@ -107,7 +113,7 @@ public class WFSTCompletionLookup extend
     }
     count = 0;
     BytesRef scratch = new BytesRef();
-    InputIterator iter = new WFSTInputIterator(iterator);
+    InputIterator iter = new WFSTInputIterator(tempDir, tempFileNamePrefix, iterator);
     IntsRefBuilder scratchInts = new IntsRefBuilder();
     BytesRefBuilder previous = null;
     PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
@@ -264,8 +270,8 @@ public class WFSTCompletionLookup extend
   
   private final class WFSTInputIterator extends SortedInputIterator {
 
-    WFSTInputIterator(InputIterator source) throws IOException {
-      super(source);
+    WFSTInputIterator(Directory tempDir, String tempFileNamePrefix, InputIterator source) throws IOException {
+      super(tempDir, tempFileNamePrefix, source);
       assert source.hasPayloads() == false;
     }
 

Modified: lucene/dev/branches/lucene6835/lucene/suggest/src/java/org/apache/lucene/search/suggest/tst/TSTLookup.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/suggest/src/java/org/apache/lucene/search/suggest/tst/TSTLookup.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/suggest/src/java/org/apache/lucene/search/suggest/tst/TSTLookup.java (original)
+++ lucene/dev/branches/lucene6835/lucene/suggest/src/java/org/apache/lucene/search/suggest/tst/TSTLookup.java Mon Oct 19 13:38:42 2015
@@ -27,11 +27,10 @@ import org.apache.lucene.search.suggest.
 import org.apache.lucene.search.suggest.SortedInputIterator;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.CharsRefBuilder;
 import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.UnicodeUtil;
 
 /**
  * Suggest implementation based on a 
@@ -45,12 +44,26 @@ public class TSTLookup extends Lookup {
 
   /** Number of entries the lookup was built with */
   private long count = 0;
+
+  private final Directory tempDir;
+  private final String tempFileNamePrefix;
   
   /** 
    * Creates a new TSTLookup with an empty Ternary Search Tree.
    * @see #build(InputIterator)
    */
-  public TSTLookup() {}
+  public TSTLookup() {
+    this(null, null);
+  }
+
+  /** 
+   * Creates a new TSTLookup, for building.
+   * @see #build(InputIterator)
+   */
+  public TSTLookup(Directory tempDir, String tempFileNamePrefix) {
+    this.tempDir = tempDir;
+    this.tempFileNamePrefix = tempFileNamePrefix;
+  }
 
   @Override
   public void build(InputIterator iterator) throws IOException {
@@ -63,7 +76,7 @@ public class TSTLookup extends Lookup {
     root = new TernaryTreeNode();
 
     // make sure it's sorted and the comparator uses UTF16 sort order
-    iterator = new SortedInputIterator(iterator, BytesRef.getUTF8SortedAsUTF16Comparator());
+    iterator = new SortedInputIterator(tempDir, tempFileNamePrefix, iterator, BytesRef.getUTF8SortedAsUTF16Comparator());
     count = 0;
     ArrayList<String> tokens = new ArrayList<>();
     ArrayList<Number> vals = new ArrayList<>();

Modified: lucene/dev/branches/lucene6835/lucene/suggest/src/test/org/apache/lucene/search/suggest/PersistenceTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/suggest/src/test/org/apache/lucene/search/suggest/PersistenceTest.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/suggest/src/test/org/apache/lucene/search/suggest/PersistenceTest.java (original)
+++ lucene/dev/branches/lucene6835/lucene/suggest/src/test/org/apache/lucene/search/suggest/PersistenceTest.java Mon Oct 19 13:38:42 2015
@@ -18,13 +18,15 @@ package org.apache.lucene.search.suggest
 
 import java.nio.file.Files;
 import java.nio.file.Path;
-import java.util.Random;
 import java.util.List;
+import java.util.Random;
 
 import org.apache.lucene.search.suggest.Lookup.LookupResult;
 import org.apache.lucene.search.suggest.fst.FSTCompletionLookup;
 import org.apache.lucene.search.suggest.jaspell.JaspellLookup;
 import org.apache.lucene.search.suggest.tst.TSTLookup;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 
@@ -58,11 +60,26 @@ public class PersistenceTest extends Luc
     runTest(FSTCompletionLookup.class, false);
   }
 
-  private void runTest(Class<? extends Lookup> lookupClass,
-      boolean supportsExactWeights) throws Exception {
+  private Directory getDirectory() {     
+    Directory dir = newDirectory();
+    if (dir instanceof MockDirectoryWrapper) {
+      ((MockDirectoryWrapper) dir).setEnableVirusScanner(false);
+    }
+    return dir;
+  }
+
+  private void runTest(Class<? extends Lookup> lookupClass, boolean supportsExactWeights) throws Exception {
 
     // Add all input keys.
-    Lookup lookup = lookupClass.newInstance();
+    Lookup lookup;
+    Directory tempDir = getDirectory();
+    if (lookupClass == TSTLookup.class) {
+      lookup = new TSTLookup(tempDir, "suggest");
+    } else if (lookupClass == FSTCompletionLookup.class) {
+      lookup = new FSTCompletionLookup(tempDir, "suggest");
+    } else {
+      lookup = lookupClass.newInstance();
+    }
     Input[] keys = new Input[this.keys.length];
     for (int i = 0; i < keys.length; i++)
       keys[i] = new Input(this.keys[i], i);
@@ -92,5 +109,6 @@ public class PersistenceTest extends Luc
         previous = lookupResult.value;
       }
     }
+    tempDir.close();
   }
 }

Modified: lucene/dev/branches/lucene6835/lucene/suggest/src/test/org/apache/lucene/search/suggest/TestInputIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/suggest/src/test/org/apache/lucene/search/suggest/TestInputIterator.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/suggest/src/test/org/apache/lucene/search/suggest/TestInputIterator.java (original)
+++ lucene/dev/branches/lucene6835/lucene/suggest/src/test/org/apache/lucene/search/suggest/TestInputIterator.java Mon Oct 19 13:38:42 2015
@@ -26,18 +26,22 @@ import java.util.Random;
 import java.util.Set;
 import java.util.TreeMap;
 
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 
 public class TestInputIterator extends LuceneTestCase {
-  
+
   public void testEmpty() throws Exception {
     InputArrayIterator iterator = new InputArrayIterator(new Input[0]);
-    InputIterator wrapper = new SortedInputIterator(iterator, BytesRef.getUTF8SortedAsUnicodeComparator());
-    assertNull(wrapper.next());
-    wrapper = new UnsortedInputIterator(iterator);
-    assertNull(wrapper.next());
+    try (Directory dir = getDirectory()) {
+      InputIterator wrapper = new SortedInputIterator(dir, "sorted", iterator, BytesRef.getUTF8SortedAsUnicodeComparator());
+      assertNull(wrapper.next());
+      wrapper = new UnsortedInputIterator(iterator);
+      assertNull(wrapper.next());
+    }
   }
   
   public void testTerms() throws Exception {
@@ -77,45 +81,51 @@ public class TestInputIterator extends L
     }
     
     // test the sorted iterator wrapper with payloads
-    InputIterator wrapper = new SortedInputIterator(new InputArrayIterator(unsorted), comparator);
-    Iterator<Map.Entry<BytesRef, SimpleEntry<Long, BytesRef>>> expected = sorted.entrySet().iterator();
-    while (expected.hasNext()) {
-      Map.Entry<BytesRef,SimpleEntry<Long, BytesRef>> entry = expected.next();
+    try (Directory tempDir = getDirectory()) {
+      InputIterator wrapper = new SortedInputIterator(tempDir, "sorted", new InputArrayIterator(unsorted), comparator);
+      Iterator<Map.Entry<BytesRef, SimpleEntry<Long, BytesRef>>> expected = sorted.entrySet().iterator();
+      while (expected.hasNext()) {
+        Map.Entry<BytesRef,SimpleEntry<Long, BytesRef>> entry = expected.next();
       
-      assertEquals(entry.getKey(), wrapper.next());
-      assertEquals(entry.getValue().getKey().longValue(), wrapper.weight());
-      assertEquals(entry.getValue().getValue(), wrapper.payload());
+        assertEquals(entry.getKey(), wrapper.next());
+        assertEquals(entry.getValue().getKey().longValue(), wrapper.weight());
+        assertEquals(entry.getValue().getValue(), wrapper.payload());
+      }
+      assertNull(wrapper.next());
     }
-    assertNull(wrapper.next());
     
     // test the sorted iterator wrapper with contexts
-    wrapper = new SortedInputIterator(new InputArrayIterator(unsortedWithContexts), comparator);
-    Iterator<Map.Entry<BytesRef, SimpleEntry<Long, Set<BytesRef>>>> actualEntries = sortedWithContext.entrySet().iterator();
-    while (actualEntries.hasNext()) {
-      Map.Entry<BytesRef, SimpleEntry<Long, Set<BytesRef>>> entry = actualEntries.next();
-      assertEquals(entry.getKey(), wrapper.next());
-      assertEquals(entry.getValue().getKey().longValue(), wrapper.weight());
-      Set<BytesRef> actualCtxs = entry.getValue().getValue();
-      assertEquals(actualCtxs, wrapper.contexts());
+    try (Directory tempDir = getDirectory()) {
+      InputIterator wrapper = new SortedInputIterator(tempDir, "sorted", new InputArrayIterator(unsortedWithContexts), comparator);
+      Iterator<Map.Entry<BytesRef, SimpleEntry<Long, Set<BytesRef>>>> actualEntries = sortedWithContext.entrySet().iterator();
+      while (actualEntries.hasNext()) {
+        Map.Entry<BytesRef, SimpleEntry<Long, Set<BytesRef>>> entry = actualEntries.next();
+        assertEquals(entry.getKey(), wrapper.next());
+        assertEquals(entry.getValue().getKey().longValue(), wrapper.weight());
+        Set<BytesRef> actualCtxs = entry.getValue().getValue();
+        assertEquals(actualCtxs, wrapper.contexts());
+      }
+      assertNull(wrapper.next());
     }
-    assertNull(wrapper.next());
-    
+
     // test the sorted iterator wrapper with contexts and payload
-    wrapper = new SortedInputIterator(new InputArrayIterator(unsortedWithPayloadAndContext), comparator);
-    Iterator<Map.Entry<BytesRef, SimpleEntry<Long, SimpleEntry<BytesRef, Set<BytesRef>>>>> expectedPayloadContextEntries = sortedWithPayloadAndContext.entrySet().iterator();
-    while (expectedPayloadContextEntries.hasNext()) {
-      Map.Entry<BytesRef, SimpleEntry<Long, SimpleEntry<BytesRef, Set<BytesRef>>>> entry = expectedPayloadContextEntries.next();
-      assertEquals(entry.getKey(), wrapper.next());
-      assertEquals(entry.getValue().getKey().longValue(), wrapper.weight());
-      Set<BytesRef> actualCtxs = entry.getValue().getValue().getValue();
-      assertEquals(actualCtxs, wrapper.contexts());
-      BytesRef actualPayload = entry.getValue().getValue().getKey();
-      assertEquals(actualPayload, wrapper.payload());
+    try (Directory tempDir = getDirectory()) {
+      InputIterator wrapper = new SortedInputIterator(tempDir, "sorter", new InputArrayIterator(unsortedWithPayloadAndContext), comparator);
+      Iterator<Map.Entry<BytesRef, SimpleEntry<Long, SimpleEntry<BytesRef, Set<BytesRef>>>>> expectedPayloadContextEntries = sortedWithPayloadAndContext.entrySet().iterator();
+      while (expectedPayloadContextEntries.hasNext()) {
+        Map.Entry<BytesRef, SimpleEntry<Long, SimpleEntry<BytesRef, Set<BytesRef>>>> entry = expectedPayloadContextEntries.next();
+        assertEquals(entry.getKey(), wrapper.next());
+        assertEquals(entry.getValue().getKey().longValue(), wrapper.weight());
+        Set<BytesRef> actualCtxs = entry.getValue().getValue().getValue();
+        assertEquals(actualCtxs, wrapper.contexts());
+        BytesRef actualPayload = entry.getValue().getValue().getKey();
+        assertEquals(actualPayload, wrapper.payload());
+      }
+      assertNull(wrapper.next());
     }
-    assertNull(wrapper.next());
-    
+
     // test the unsorted iterator wrapper with payloads
-    wrapper = new UnsortedInputIterator(new InputArrayIterator(unsorted));
+    InputIterator wrapper = new UnsortedInputIterator(new InputArrayIterator(unsorted));
     TreeMap<BytesRef, SimpleEntry<Long, BytesRef>> actual = new TreeMap<>();
     BytesRef key;
     while ((key = wrapper.next()) != null) {
@@ -126,19 +136,21 @@ public class TestInputIterator extends L
     assertEquals(sorted, actual);
 
     // test the sorted iterator wrapper without payloads
-    InputIterator wrapperWithoutPayload = new SortedInputIterator(new InputArrayIterator(unsortedWithoutPayload), comparator);
-    Iterator<Map.Entry<BytesRef, Long>> expectedWithoutPayload = sortedWithoutPayload.entrySet().iterator();
-    while (expectedWithoutPayload.hasNext()) {
-      Map.Entry<BytesRef, Long> entry = expectedWithoutPayload.next();
+    try (Directory tempDir = getDirectory()) {
+      InputIterator wrapperWithoutPayload = new SortedInputIterator(tempDir, "sorted", new InputArrayIterator(unsortedWithoutPayload), comparator);
+      Iterator<Map.Entry<BytesRef, Long>> expectedWithoutPayload = sortedWithoutPayload.entrySet().iterator();
+      while (expectedWithoutPayload.hasNext()) {
+        Map.Entry<BytesRef, Long> entry = expectedWithoutPayload.next();
       
-      assertEquals(entry.getKey(), wrapperWithoutPayload.next());
-      assertEquals(entry.getValue().longValue(), wrapperWithoutPayload.weight());
-      assertNull(wrapperWithoutPayload.payload());
+        assertEquals(entry.getKey(), wrapperWithoutPayload.next());
+        assertEquals(entry.getValue().longValue(), wrapperWithoutPayload.weight());
+        assertNull(wrapperWithoutPayload.payload());
+      }
+      assertNull(wrapperWithoutPayload.next());
     }
-    assertNull(wrapperWithoutPayload.next());
     
     // test the unsorted iterator wrapper without payloads
-    wrapperWithoutPayload = new UnsortedInputIterator(new InputArrayIterator(unsortedWithoutPayload));
+    InputIterator wrapperWithoutPayload = new UnsortedInputIterator(new InputArrayIterator(unsortedWithoutPayload));
     TreeMap<BytesRef, Long> actualWithoutPayload = new TreeMap<>();
     while ((key = wrapperWithoutPayload.next()) != null) {
       long value = wrapperWithoutPayload.weight();
@@ -157,4 +169,12 @@ public class TestInputIterator extends L
     return ((b.bytes[pos++] & 0xFF) << 24) | ((b.bytes[pos++] & 0xFF) << 16)
         | ((b.bytes[pos++] & 0xFF) << 8) | (b.bytes[pos] & 0xFF);
   }
+
+  private Directory getDirectory() {     
+    Directory dir = newDirectory();
+    if (dir instanceof MockDirectoryWrapper) {
+      ((MockDirectoryWrapper) dir).setEnableVirusScanner(false);
+    }
+    return dir;
+  }
 }



Mime
View raw message