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 11:58:19 UTC

svn commit: r1708760 [3/4] - in /lucene/dev/trunk: lucene/ lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/ lucene/analysis/common/src/test/org/apache/lucene/analysis/core/ lucene/analysis/common/src/test/org/apache/lucene/analysis/...

Modified: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java?rev=1708760&r1=1708759&r2=1708760&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java Thu Oct 15 09:58:18 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/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java?rev=1708760&r1=1708759&r2=1708760&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java Thu Oct 15 09:58:18 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/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java?rev=1708760&r1=1708759&r2=1708760&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java Thu Oct 15 09:58:18 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/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java?rev=1708760&r1=1708759&r2=1708760&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java Thu Oct 15 09:58:18 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/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java?rev=1708760&r1=1708759&r2=1708760&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java Thu Oct 15 09:58:18 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/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/tst/TSTLookup.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/tst/TSTLookup.java?rev=1708760&r1=1708759&r2=1708760&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/tst/TSTLookup.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/tst/TSTLookup.java Thu Oct 15 09:58:18 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/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/PersistenceTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/PersistenceTest.java?rev=1708760&r1=1708759&r2=1708760&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/PersistenceTest.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/PersistenceTest.java Thu Oct 15 09:58:18 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/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/TestInputIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/TestInputIterator.java?rev=1708760&r1=1708759&r2=1708760&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/TestInputIterator.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/TestInputIterator.java Thu Oct 15 09:58:18 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;
+  }
 }

Modified: lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggesterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggesterTest.java?rev=1708760&r1=1708759&r2=1708760&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggesterTest.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggesterTest.java Thu Oct 15 09:58:18 2015
@@ -34,8 +34,8 @@ import java.util.Set;
 import java.util.TreeSet;
 
 import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.CannedBinaryTokenStream.BinaryToken;
 import org.apache.lucene.analysis.CannedBinaryTokenStream;
+import org.apache.lucene.analysis.CannedBinaryTokenStream.BinaryToken;
 import org.apache.lucene.analysis.CannedTokenStream;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenFilter;
@@ -48,9 +48,11 @@ import org.apache.lucene.analysis.Tokeni
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.search.suggest.Lookup.LookupResult;
 import org.apache.lucene.search.suggest.Input;
 import org.apache.lucene.search.suggest.InputArrayIterator;
+import org.apache.lucene.search.suggest.Lookup.LookupResult;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LineFileDocs;
@@ -58,7 +60,7 @@ import org.apache.lucene.util.LuceneTest
 import org.apache.lucene.util.TestUtil;
 
 public class AnalyzingSuggesterTest extends LuceneTestCase {
-  
+
   /** this is basically the WFST test ported to KeywordAnalyzer. so it acts the same */
   public void testKeyword() throws Exception {
     Iterable<Input> keys = shuffle(
@@ -71,8 +73,10 @@ public class AnalyzingSuggesterTest exte
         new Input("barbara", 1)
     );
 
+    Directory tempDir = getDirectory();
+
     Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.KEYWORD, false);
-    AnalyzingSuggester suggester = new AnalyzingSuggester(analyzer);
+    AnalyzingSuggester suggester = new AnalyzingSuggester(tempDir, "suggest", analyzer);
     suggester.build(new InputArrayIterator(keys));
     
     // top N of 2, but only foo is available
@@ -106,7 +110,7 @@ public class AnalyzingSuggesterTest exte
     assertEquals("barbara", results.get(2).key.toString());
     assertEquals(6, results.get(2).value, 0.01F);
     
-    analyzer.close();
+    IOUtils.close(analyzer, tempDir);
   }
   
   public void testKeywordWithPayloads() throws Exception {
@@ -119,7 +123,8 @@ public class AnalyzingSuggesterTest exte
       new Input("barbara", 6, new BytesRef("for all the fish")));
     
     Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.KEYWORD, false);
-    AnalyzingSuggester suggester = new AnalyzingSuggester(analyzer);
+    Directory tempDir = getDirectory();
+    AnalyzingSuggester suggester = new AnalyzingSuggester(tempDir, "suggest", analyzer);
     suggester.build(new InputArrayIterator(keys));
     for (int i = 0; i < 2; i++) {
       // top N of 2, but only foo is available
@@ -160,7 +165,7 @@ public class AnalyzingSuggesterTest exte
       assertEquals(6, results.get(2).value, 0.01F);
       assertEquals(new BytesRef("for all the fish"), results.get(2).payload);
     }
-    analyzer.close();
+    IOUtils.close(analyzer, tempDir);
   }
   
   public void testRandomRealisticKeys() throws IOException {
@@ -180,7 +185,9 @@ public class AnalyzingSuggesterTest exte
     }
     Analyzer indexAnalyzer = new MockAnalyzer(random());
     Analyzer queryAnalyzer = new MockAnalyzer(random());
-    AnalyzingSuggester analyzingSuggester = new AnalyzingSuggester(indexAnalyzer, queryAnalyzer,
+    Directory tempDir = getDirectory();
+
+    AnalyzingSuggester analyzingSuggester = new AnalyzingSuggester(tempDir, "suggest", indexAnalyzer, queryAnalyzer,
         AnalyzingSuggester.EXACT_FIRST | AnalyzingSuggester.PRESERVE_SEP, 256, -1, random().nextBoolean());
     boolean doPayloads = random().nextBoolean();
     if (doPayloads) {
@@ -205,7 +212,7 @@ public class AnalyzingSuggesterTest exte
       }
     }
     
-    IOUtils.close(lineFile, indexAnalyzer, queryAnalyzer);
+    IOUtils.close(lineFile, indexAnalyzer, queryAnalyzer, tempDir);
   }
   
   // TODO: more tests
@@ -217,8 +224,9 @@ public class AnalyzingSuggesterTest exte
         new Input("the ghost of christmas past", 50),
     };
     
+    Directory tempDir = getDirectory();
     Analyzer standard = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, true, MockTokenFilter.ENGLISH_STOPSET);
-    AnalyzingSuggester suggester = new AnalyzingSuggester(standard, standard, 
+    AnalyzingSuggester suggester = new AnalyzingSuggester(tempDir, "suggest", standard, standard, 
         AnalyzingSuggester.EXACT_FIRST | AnalyzingSuggester.PRESERVE_SEP, 256, -1, false);
 
     suggester.build(new InputArrayIterator(keys));
@@ -240,17 +248,18 @@ public class AnalyzingSuggesterTest exte
     assertEquals("the ghost of christmas past", results.get(0).key.toString());
     assertEquals(50, results.get(0).value, 0.01F);
     
-    standard.close();
+    IOUtils.close(standard, tempDir);
   }
 
   public void testEmpty() throws Exception {
     Analyzer standard = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, true, MockTokenFilter.ENGLISH_STOPSET);
-    AnalyzingSuggester suggester = new AnalyzingSuggester(standard);
+    Directory tempDir = getDirectory();
+    AnalyzingSuggester suggester = new AnalyzingSuggester(tempDir, "suggest", standard);
     suggester.build(new InputArrayIterator(new Input[0]));
 
     List<LookupResult> result = suggester.lookup("a", false, 20);
     assertTrue(result.isEmpty());
-    standard.close();
+    IOUtils.close(standard, tempDir);
   }
 
   public void testNoSeps() throws Exception {
@@ -262,7 +271,8 @@ public class AnalyzingSuggesterTest exte
     int options = 0;
 
     Analyzer a = new MockAnalyzer(random());
-    AnalyzingSuggester suggester = new AnalyzingSuggester(a, a, options, 256, -1, true);
+    Directory tempDir = getDirectory();
+    AnalyzingSuggester suggester = new AnalyzingSuggester(tempDir, "suggest", a, a, options, 256, -1, true);
     suggester.build(new InputArrayIterator(keys));
     // TODO: would be nice if "ab " would allow the test to
     // pass, and more generally if the analyzer can know
@@ -275,7 +285,7 @@ public class AnalyzingSuggesterTest exte
     // complete to "abcd", which has higher weight so should
     // appear first:
     assertEquals("abcd", r.get(0).key.toString());
-    a.close();
+    IOUtils.close(a, tempDir);
   }
 
   public void testGraphDups() throws Exception {
@@ -330,7 +340,8 @@ public class AnalyzingSuggesterTest exte
         new Input("wi fi network is fast", 10),
     };
     //AnalyzingSuggester suggester = new AnalyzingSuggester(analyzer, AnalyzingSuggester.EXACT_FIRST, 256, -1);
-    AnalyzingSuggester suggester = new AnalyzingSuggester(analyzer);
+    Directory tempDir = getDirectory();
+    AnalyzingSuggester suggester = new AnalyzingSuggester(tempDir, "suggest", analyzer);
     suggester.build(new InputArrayIterator(keys));
     List<LookupResult> results = suggester.lookup("wifi network", false, 10);
     if (VERBOSE) {
@@ -341,7 +352,7 @@ public class AnalyzingSuggesterTest exte
     assertEquals(50, results.get(0).value);
     assertEquals("wi fi network is fast", results.get(1).key);
     assertEquals(10, results.get(1).value);
-    analyzer.close();
+    IOUtils.close(analyzer, tempDir);
   }
 
   public void testInputPathRequired() throws Exception {
@@ -396,11 +407,12 @@ public class AnalyzingSuggesterTest exte
         new Input("ab xc", 50),
         new Input("ba xd", 50),
     };
-    AnalyzingSuggester suggester = new AnalyzingSuggester(analyzer);
+    Directory tempDir = getDirectory();
+    AnalyzingSuggester suggester = new AnalyzingSuggester(tempDir, "suggest", analyzer);
     suggester.build(new InputArrayIterator(keys));
     List<LookupResult> results = suggester.lookup("ab x", false, 1);
     assertTrue(results.size() == 1);
-    analyzer.close();
+    IOUtils.close(analyzer, tempDir);
   }
 
   private static Token token(String term, int posInc, int posLength) {
@@ -471,7 +483,8 @@ public class AnalyzingSuggesterTest exte
 
     Analyzer a = getUnusualAnalyzer();
     int options = AnalyzingSuggester.EXACT_FIRST | AnalyzingSuggester.PRESERVE_SEP;
-    AnalyzingSuggester suggester = new AnalyzingSuggester(a, a, options, 256, -1, true);
+    Directory tempDir = getDirectory();
+    AnalyzingSuggester suggester = new AnalyzingSuggester(tempDir, "suggest", a, a, options, 256, -1, true);
     suggester.build(new InputArrayIterator(new Input[] {
           new Input("x y", 1),
           new Input("x y z", 3),
@@ -505,13 +518,14 @@ public class AnalyzingSuggesterTest exte
         }
       }
     }
-    a.close();
+    IOUtils.close(a, tempDir);
   }
 
   public void testNonExactFirst() throws Exception {
 
     Analyzer a = getUnusualAnalyzer();
-    AnalyzingSuggester suggester = new AnalyzingSuggester(a, a, AnalyzingSuggester.PRESERVE_SEP, 256, -1, true);
+    Directory tempDir = getDirectory();
+    AnalyzingSuggester suggester = new AnalyzingSuggester(tempDir, "suggest", a, a, AnalyzingSuggester.PRESERVE_SEP, 256, -1, true);
 
     suggester.build(new InputArrayIterator(new Input[] {
           new Input("x y", 1),
@@ -543,7 +557,7 @@ public class AnalyzingSuggesterTest exte
         }
       }
     }
-    a.close();
+    IOUtils.close(a, tempDir);
   }
   
   // Holds surface form separately:
@@ -764,7 +778,8 @@ public class AnalyzingSuggesterTest exte
     }
 
     Analyzer a = new MockTokenEatingAnalyzer(numStopChars, preserveHoles);
-    AnalyzingSuggester suggester = new AnalyzingSuggester(a, a,
+    Directory tempDir = getDirectory();
+    AnalyzingSuggester suggester = new AnalyzingSuggester(tempDir, "suggest", a, a,
                                                           preserveSep ? AnalyzingSuggester.PRESERVE_SEP : 0, 256, -1, true);
     if (doPayloads) {
       suggester.build(new InputArrayIterator(shuffle(payloadKeys)));
@@ -882,12 +897,13 @@ public class AnalyzingSuggesterTest exte
         }
       }
     }
-    a.close();
+    IOUtils.close(a, tempDir);
   }
 
   public void testMaxSurfaceFormsPerAnalyzedForm() throws Exception {
     Analyzer a = new MockAnalyzer(random());
-    AnalyzingSuggester suggester = new AnalyzingSuggester(a, a, 0, 2, -1, true);
+    Directory tempDir = getDirectory();
+    AnalyzingSuggester suggester = new AnalyzingSuggester(tempDir, "suggest", a, a, 0, 2, -1, true);
     suggester.build(new InputArrayIterator(shuffle(new Input("a", 40),
         new Input("a ", 50), new Input(" a", 60))));
 
@@ -897,12 +913,13 @@ public class AnalyzingSuggesterTest exte
     assertEquals(60, results.get(0).value);
     assertEquals("a ", results.get(1).key);
     assertEquals(50, results.get(1).value);
-    a.close();
+    IOUtils.close(a, tempDir);
   }
 
   public void testQueueExhaustion() throws Exception {
     Analyzer a = new MockAnalyzer(random());
-    AnalyzingSuggester suggester = new AnalyzingSuggester(a, a, AnalyzingSuggester.EXACT_FIRST, 256, -1, true);
+    Directory tempDir = getDirectory();
+    AnalyzingSuggester suggester = new AnalyzingSuggester(tempDir, "suggest", a, a, AnalyzingSuggester.EXACT_FIRST, 256, -1, true);
 
     suggester.build(new InputArrayIterator(new Input[] {
           new Input("a", 2),
@@ -912,14 +929,15 @@ public class AnalyzingSuggesterTest exte
         }));
 
     suggester.lookup("a", false, 4);
-    a.close();
+    IOUtils.close(a, tempDir);
   }
 
   public void testExactFirstMissingResult() throws Exception {
 
     Analyzer a = new MockAnalyzer(random());
 
-    AnalyzingSuggester suggester = new AnalyzingSuggester(a, a, AnalyzingSuggester.EXACT_FIRST, 256, -1, true);
+    Directory tempDir = getDirectory();
+    AnalyzingSuggester suggester = new AnalyzingSuggester(tempDir, "suggest", a, a, AnalyzingSuggester.EXACT_FIRST, 256, -1, true);
 
     suggester.build(new InputArrayIterator(new Input[] {
           new Input("a", 5),
@@ -959,7 +977,7 @@ public class AnalyzingSuggesterTest exte
     assertEquals(4, results.get(1).value);
     assertEquals("a b", results.get(2).key);
     assertEquals(3, results.get(2).value);
-    a.close();
+    IOUtils.close(a, tempDir);
   }
 
   public void testDupSurfaceFormsMissingResults() throws Exception {
@@ -986,7 +1004,8 @@ public class AnalyzingSuggesterTest exte
       }
     };
 
-    AnalyzingSuggester suggester = new AnalyzingSuggester(a, a, 0, 256, -1, true);
+    Directory tempDir = getDirectory();
+    AnalyzingSuggester suggester = new AnalyzingSuggester(tempDir, "suggest", a, a, 0, 256, -1, true);
 
     suggester.build(new InputArrayIterator(shuffle(
           new Input("hambone", 6),
@@ -1018,7 +1037,7 @@ public class AnalyzingSuggesterTest exte
     assertEquals(6, results.get(0).value);
     assertEquals("nellie", results.get(1).key);
     assertEquals(5, results.get(1).value);
-    a.close();
+    IOUtils.close(a, tempDir);
   }
 
   public void testDupSurfaceFormsMissingResults2() throws Exception {
@@ -1055,7 +1074,8 @@ public class AnalyzingSuggesterTest exte
       }
     };
 
-    AnalyzingSuggester suggester = new AnalyzingSuggester(a, a, 0, 256, -1, true);
+    Directory tempDir = getDirectory();
+    AnalyzingSuggester suggester = new AnalyzingSuggester(tempDir, "suggest", a, a, 0, 256, -1, true);
 
     suggester.build(new InputArrayIterator(new Input[] {
           new Input("a", 6),
@@ -1088,7 +1108,7 @@ public class AnalyzingSuggesterTest exte
     assertEquals(6, results.get(0).value);
     assertEquals("b", results.get(1).key);
     assertEquals(5, results.get(1).value);
-    a.close();
+    IOUtils.close(a, tempDir);
   }
 
   public void test0ByteKeys() throws Exception {
@@ -1128,19 +1148,21 @@ public class AnalyzingSuggesterTest exte
         }
       };
 
-    AnalyzingSuggester suggester = new AnalyzingSuggester(a, a, 0, 256, -1, true);
+    Directory tempDir = getDirectory();
+    AnalyzingSuggester suggester = new AnalyzingSuggester(tempDir, "suggest", a, a, 0, 256, -1, true);
 
     suggester.build(new InputArrayIterator(new Input[] {
           new Input("a a", 50),
           new Input("a b", 50),
         }));
     
-    a.close();
+    IOUtils.close(a, tempDir);
   }
 
   public void testDupSurfaceFormsMissingResults3() throws Exception {
     Analyzer a = new MockAnalyzer(random());
-    AnalyzingSuggester suggester = new AnalyzingSuggester(a, a, AnalyzingSuggester.PRESERVE_SEP, 256, -1, true);
+    Directory tempDir = getDirectory();
+    AnalyzingSuggester suggester = new AnalyzingSuggester(tempDir, "suggest", a, a, AnalyzingSuggester.PRESERVE_SEP, 256, -1, true);
     suggester.build(new InputArrayIterator(new Input[] {
           new Input("a a", 7),
           new Input("a a", 7),
@@ -1149,19 +1171,20 @@ public class AnalyzingSuggesterTest exte
           new Input("a b", 5),
         }));
     assertEquals("[a a/7, a c/6, a b/5]", suggester.lookup("a", false, 3).toString());
-    a.close();
+    IOUtils.close(tempDir, a);
   }
 
   public void testEndingSpace() throws Exception {
     Analyzer a = new MockAnalyzer(random());
-    AnalyzingSuggester suggester = new AnalyzingSuggester(a, a, AnalyzingSuggester.PRESERVE_SEP, 256, -1, true);
+    Directory tempDir = getDirectory();
+    AnalyzingSuggester suggester = new AnalyzingSuggester(tempDir, "suggest", a, a, AnalyzingSuggester.PRESERVE_SEP, 256, -1, true);
     suggester.build(new InputArrayIterator(new Input[] {
           new Input("i love lucy", 7),
           new Input("isla de muerta", 8),
         }));
     assertEquals("[isla de muerta/8, i love lucy/7]", suggester.lookup("i", false, 3).toString());
     assertEquals("[i love lucy/7]", suggester.lookup("i ", false, 3).toString());
-    a.close();
+    IOUtils.close(a, tempDir);
   }
 
   public void testTooManyExpansions() throws Exception {
@@ -1188,15 +1211,17 @@ public class AnalyzingSuggesterTest exte
         }
       };
 
-    AnalyzingSuggester suggester = new AnalyzingSuggester(a, a, 0, 256, 1, true);
+    Directory tempDir = getDirectory();
+    AnalyzingSuggester suggester = new AnalyzingSuggester(tempDir, "suggest", a, a, 0, 256, 1, true);
     suggester.build(new InputArrayIterator(new Input[] {new Input("a", 1)}));
     assertEquals("[a/1]", suggester.lookup("a", false, 1).toString());
-    a.close();
+    IOUtils.close(a, tempDir);
   }
   
   public void testIllegalLookupArgument() throws Exception {
     Analyzer a = new MockAnalyzer(random());
-    AnalyzingSuggester suggester = new AnalyzingSuggester(a, a, 0, 256, -1, true);
+    Directory tempDir = getDirectory();
+    AnalyzingSuggester suggester = new AnalyzingSuggester(tempDir, "suggest", a, a, 0, 256, -1, true);
     suggester.build(new InputArrayIterator(new Input[] {
         new Input("а где Люси?", 7),
     }));
@@ -1212,7 +1237,7 @@ public class AnalyzingSuggesterTest exte
     } catch (IllegalArgumentException e) {
       // expected
     }
-    a.close();
+    IOUtils.close(a, tempDir);
   }
 
   static final Iterable<Input> shuffle(Input...values) {
@@ -1227,7 +1252,8 @@ public class AnalyzingSuggesterTest exte
   // TODO: we need BaseSuggesterTestCase?
   public void testTooLongSuggestion() throws Exception {
     Analyzer a = new MockAnalyzer(random());
-    AnalyzingSuggester suggester = new AnalyzingSuggester(a);
+    Directory tempDir = getDirectory();
+    AnalyzingSuggester suggester = new AnalyzingSuggester(tempDir, "suggest", a);
     String bigString = TestUtil.randomSimpleString(random(), 30000, 30000);
     try {
       suggester.build(new InputArrayIterator(new Input[] {
@@ -1238,6 +1264,14 @@ public class AnalyzingSuggesterTest exte
     } catch (IllegalArgumentException iae) {
       // expected
     }
-    a.close();
+    IOUtils.close(a, tempDir);
+  }
+
+  private Directory getDirectory() {     
+    Directory dir = newDirectory();
+    if (dir instanceof MockDirectoryWrapper) {
+      ((MockDirectoryWrapper) dir).setEnableVirusScanner(false);
+    }
+    return dir;
   }
 }

Modified: lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/FuzzySuggesterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/FuzzySuggesterTest.java?rev=1708760&r1=1708759&r2=1708760&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/FuzzySuggesterTest.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/FuzzySuggesterTest.java Thu Oct 15 09:58:18 2015
@@ -43,8 +43,11 @@ import org.apache.lucene.analysis.tokena
 import org.apache.lucene.search.suggest.Input;
 import org.apache.lucene.search.suggest.InputArrayIterator;
 import org.apache.lucene.search.suggest.Lookup.LookupResult;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
@@ -62,7 +65,8 @@ public class FuzzySuggesterTest extends
     }
     keys.add(new Input("foo bar boo far", 12));
     MockAnalyzer analyzer = new MockAnalyzer(random(), MockTokenizer.KEYWORD, false);
-    FuzzySuggester suggester = new FuzzySuggester(analyzer, analyzer, FuzzySuggester.EXACT_FIRST | FuzzySuggester.PRESERVE_SEP, 256, -1, true, FuzzySuggester.DEFAULT_MAX_EDITS, FuzzySuggester.DEFAULT_TRANSPOSITIONS,
+    Directory tempDir = getDirectory();
+    FuzzySuggester suggester = new FuzzySuggester(tempDir, "fuzzy", analyzer, analyzer, FuzzySuggester.EXACT_FIRST | FuzzySuggester.PRESERVE_SEP, 256, -1, true, FuzzySuggester.DEFAULT_MAX_EDITS, FuzzySuggester.DEFAULT_TRANSPOSITIONS,
                                                   0, FuzzySuggester.DEFAULT_MIN_FUZZY_LENGTH, FuzzySuggester.DEFAULT_UNICODE_AWARE);
     suggester.build(new InputArrayIterator(keys));
     int numIters = atLeast(10);
@@ -73,7 +77,7 @@ public class FuzzySuggesterTest extends
       assertEquals("foo bar boo far", results.get(0).key.toString());
       assertEquals(12, results.get(0).value, 0.01F);  
     }
-    analyzer.close();
+    IOUtils.close(analyzer, tempDir);
   }
   
   public void testNonLatinRandomEdits() throws IOException {
@@ -84,7 +88,8 @@ public class FuzzySuggesterTest extends
     }
     keys.add(new Input("фуу бар буу фар", 12));
     MockAnalyzer analyzer = new MockAnalyzer(random(), MockTokenizer.KEYWORD, false);
-    FuzzySuggester suggester = new FuzzySuggester(analyzer, analyzer, FuzzySuggester.EXACT_FIRST | FuzzySuggester.PRESERVE_SEP, 256, -1, true, FuzzySuggester.DEFAULT_MAX_EDITS, FuzzySuggester.DEFAULT_TRANSPOSITIONS,
+    Directory tempDir = getDirectory();
+    FuzzySuggester suggester = new FuzzySuggester(tempDir, "fuzzy",analyzer, analyzer, FuzzySuggester.EXACT_FIRST | FuzzySuggester.PRESERVE_SEP, 256, -1, true, FuzzySuggester.DEFAULT_MAX_EDITS, FuzzySuggester.DEFAULT_TRANSPOSITIONS,
         0, FuzzySuggester.DEFAULT_MIN_FUZZY_LENGTH, true);
     suggester.build(new InputArrayIterator(keys));
     int numIters = atLeast(10);
@@ -95,7 +100,7 @@ public class FuzzySuggesterTest extends
       assertEquals("фуу бар буу фар", results.get(0).key.toString());
       assertEquals(12, results.get(0).value, 0.01F);
     }
-    analyzer.close();
+    IOUtils.close(analyzer, tempDir);
   }
 
   /** this is basically the WFST test ported to KeywordAnalyzer. so it acts the same */
@@ -108,7 +113,8 @@ public class FuzzySuggesterTest extends
     };
     
     Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.KEYWORD, false);
-    FuzzySuggester suggester = new FuzzySuggester(analyzer);
+    Directory tempDir = getDirectory();
+    FuzzySuggester suggester = new FuzzySuggester(tempDir, "fuzzy",analyzer);
     suggester.build(new InputArrayIterator(keys));
     
     List<LookupResult> results = suggester.lookup(TestUtil.stringToCharSequence("bariar", random()), false, 2);
@@ -171,7 +177,7 @@ public class FuzzySuggesterTest extends
     assertEquals("barbara", results.get(2).key.toString());
     assertEquals(6, results.get(2).value, 0.01F);
     
-    analyzer.close();
+    IOUtils.close(analyzer, tempDir);
   }
   
   /**
@@ -183,7 +189,8 @@ public class FuzzySuggesterTest extends
     };
     
     Analyzer standard = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, true, MockTokenFilter.ENGLISH_STOPSET);
-    FuzzySuggester suggester = new FuzzySuggester(standard, standard, AnalyzingSuggester.EXACT_FIRST | AnalyzingSuggester.PRESERVE_SEP, 256, -1, false, FuzzySuggester.DEFAULT_MAX_EDITS, FuzzySuggester.DEFAULT_TRANSPOSITIONS,
+    Directory tempDir = getDirectory();
+    FuzzySuggester suggester = new FuzzySuggester(tempDir, "fuzzy", standard, standard, AnalyzingSuggester.EXACT_FIRST | AnalyzingSuggester.PRESERVE_SEP, 256, -1, false, FuzzySuggester.DEFAULT_MAX_EDITS, FuzzySuggester.DEFAULT_TRANSPOSITIONS,
         FuzzySuggester.DEFAULT_NON_FUZZY_PREFIX, FuzzySuggester.DEFAULT_MIN_FUZZY_LENGTH, FuzzySuggester.DEFAULT_UNICODE_AWARE);
     suggester.build(new InputArrayIterator(keys));
     
@@ -204,7 +211,7 @@ public class FuzzySuggesterTest extends
     assertEquals("the ghost of christmas past", results.get(0).key.toString());
     assertEquals(50, results.get(0).value, 0.01F);
     
-    standard.close();
+    IOUtils.close(standard, tempDir);
   }
 
   public void testNoSeps() throws Exception {
@@ -216,7 +223,8 @@ public class FuzzySuggesterTest extends
     int options = 0;
 
     Analyzer a = new MockAnalyzer(random());
-    FuzzySuggester suggester = new FuzzySuggester(a, a, options, 256, -1, true, 1, true, 1, 3, false);
+    Directory tempDir = getDirectory();
+    FuzzySuggester suggester = new FuzzySuggester(tempDir, "fuzzy",a, a, options, 256, -1, true, 1, true, 1, 3, false);
     suggester.build(new InputArrayIterator(keys));
     // TODO: would be nice if "ab " would allow the test to
     // pass, and more generally if the analyzer can know
@@ -229,7 +237,7 @@ public class FuzzySuggesterTest extends
     // complete to "abcd", which has higher weight so should
     // appear first:
     assertEquals("abcd", r.get(0).key.toString());
-    a.close();
+    IOUtils.close(a, tempDir);
   }
 
   public void testGraphDups() throws Exception {
@@ -283,7 +291,8 @@ public class FuzzySuggesterTest extends
         new Input("wifi network is slow", 50),
         new Input("wi fi network is fast", 10),
     };
-    FuzzySuggester suggester = new FuzzySuggester(analyzer);
+    Directory tempDir = getDirectory();
+    FuzzySuggester suggester = new FuzzySuggester(tempDir, "fuzzy", analyzer);
     suggester.build(new InputArrayIterator(keys));
     
     List<LookupResult> results = suggester.lookup("wifi network", false, 10);
@@ -295,17 +304,18 @@ public class FuzzySuggesterTest extends
     assertEquals(50, results.get(0).value);
     assertEquals("wi fi network is fast", results.get(1).key);
     assertEquals(10, results.get(1).value);
-    analyzer.close();
+    IOUtils.close(tempDir, analyzer);
   }
 
   public void testEmpty() throws Exception {
     Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.KEYWORD, false);
-    FuzzySuggester suggester = new FuzzySuggester(analyzer);
+    Directory tempDir = getDirectory();
+    FuzzySuggester suggester = new FuzzySuggester(tempDir, "fuzzy", analyzer);
     suggester.build(new InputArrayIterator(new Input[0]));
 
     List<LookupResult> result = suggester.lookup("a", false, 20);
     assertTrue(result.isEmpty());
-    analyzer.close();
+    IOUtils.close(analyzer, tempDir);
   }
 
   public void testInputPathRequired() throws Exception {
@@ -360,11 +370,12 @@ public class FuzzySuggesterTest extends
         new Input("ab xc", 50),
         new Input("ba xd", 50),
     };
-    FuzzySuggester suggester = new FuzzySuggester(analyzer);
+    Directory tempDir = getDirectory();
+    FuzzySuggester suggester = new FuzzySuggester(tempDir, "fuzzy", analyzer);
     suggester.build(new InputArrayIterator(keys));
     List<LookupResult> results = suggester.lookup("ab x", false, 1);
     assertTrue(results.size() == 1);
-    analyzer.close();
+    IOUtils.close(analyzer, tempDir);
   }
 
   private static Token token(String term, int posInc, int posLength) {
@@ -430,7 +441,8 @@ public class FuzzySuggesterTest extends
   public void testExactFirst() throws Exception {
 
     Analyzer a = getUnusualAnalyzer();
-    FuzzySuggester suggester = new FuzzySuggester(a, a, AnalyzingSuggester.EXACT_FIRST | AnalyzingSuggester.PRESERVE_SEP, 256, -1, true, 1, true, 1, 3, false);
+    Directory tempDir = getDirectory();
+    FuzzySuggester suggester = new FuzzySuggester(tempDir, "fuzzy", a, a, AnalyzingSuggester.EXACT_FIRST | AnalyzingSuggester.PRESERVE_SEP, 256, -1, true, 1, true, 1, 3, false);
     suggester.build(new InputArrayIterator(new Input[] {
           new Input("x y", 1),
           new Input("x y z", 3),
@@ -464,13 +476,14 @@ public class FuzzySuggesterTest extends
         }
       }
     }
-    a.close();
+    IOUtils.close(a, tempDir);
   }
 
   public void testNonExactFirst() throws Exception {
 
     Analyzer a = getUnusualAnalyzer();
-    FuzzySuggester suggester = new FuzzySuggester(a, a, AnalyzingSuggester.PRESERVE_SEP, 256, -1, true, 1, true, 1, 3, false);
+    Directory tempDir = getDirectory();
+    FuzzySuggester suggester = new FuzzySuggester(tempDir, "fuzzy", a, a, AnalyzingSuggester.PRESERVE_SEP, 256, -1, true, 1, true, 1, 3, false);
 
     suggester.build(new InputArrayIterator(new Input[] {
           new Input("x y", 1),
@@ -502,7 +515,7 @@ public class FuzzySuggesterTest extends
         }
       }
     }
-    a.close();
+    IOUtils.close(a, tempDir);
   }
   
   // Holds surface form separately:
@@ -697,7 +710,8 @@ public class FuzzySuggesterTest extends
     }
 
     Analyzer a = new MockTokenEatingAnalyzer(numStopChars, preserveHoles);
-    FuzzySuggester suggester = new FuzzySuggester(a, a,
+    Directory tempDir = getDirectory();
+    FuzzySuggester suggester = new FuzzySuggester(tempDir, "fuzzy",a, a,
                                                   preserveSep ? AnalyzingSuggester.PRESERVE_SEP : 0, 256, -1, true, 1, false, 1, 3, unicodeAware);
     suggester.build(new InputArrayIterator(keys));
 
@@ -836,12 +850,13 @@ public class FuzzySuggesterTest extends
         assertEquals(matches.get(hit).value, r.get(hit).value, 0f);
       }
     }
-    a.close();
+    IOUtils.close(a, tempDir);
   }
 
   public void testMaxSurfaceFormsPerAnalyzedForm() throws Exception {
     Analyzer a = new MockAnalyzer(random());
-    FuzzySuggester suggester = new FuzzySuggester(a, a, 0, 2, -1, true, 1, true, 1, 3, false);
+    Directory tempDir = getDirectory();
+    FuzzySuggester suggester = new FuzzySuggester(tempDir, "fuzzy", a, a, 0, 2, -1, true, 1, true, 1, 3, false);
 
     List<Input> keys = Arrays.asList(new Input[] {
         new Input("a", 40),
@@ -858,12 +873,13 @@ public class FuzzySuggesterTest extends
     assertEquals(60, results.get(0).value);
     assertEquals("a ", results.get(1).key);
     assertEquals(50, results.get(1).value);
-    a.close();
+    IOUtils.close(a, tempDir);
   }
 
   public void testEditSeps() throws Exception {
     Analyzer a = new MockAnalyzer(random());
-    FuzzySuggester suggester = new FuzzySuggester(a, a, FuzzySuggester.PRESERVE_SEP, 2, -1, true, 2, true, 1, 3, false);
+    Directory tempDir = getDirectory();
+    FuzzySuggester suggester = new FuzzySuggester(tempDir, "fuzzy", a, a, FuzzySuggester.PRESERVE_SEP, 2, -1, true, 2, true, 1, 3, false);
 
     List<Input> keys = Arrays.asList(new Input[] {
         new Input("foo bar", 40),
@@ -879,7 +895,7 @@ public class FuzzySuggesterTest extends
     assertEquals("[foo bar baz/50]", suggester.lookup("foobarbaz", false, 5).toString());
     assertEquals("[barbaz/60, barbazfoo/10]", suggester.lookup("bar baz", false, 5).toString());
     assertEquals("[barbazfoo/10]", suggester.lookup("bar baz foo", false, 5).toString());
-    a.close();
+    IOUtils.close(a, tempDir);
   }
   
   @SuppressWarnings("fallthrough")
@@ -978,7 +994,8 @@ public class FuzzySuggesterTest extends
     boolean transpositions = random().nextBoolean();
     // TODO: test graph analyzers
     // TODO: test exactFirst / preserveSep permutations
-    FuzzySuggester suggest = new FuzzySuggester(a, a, 0, 256, -1, true, maxEdits, transpositions, prefixLen, prefixLen, false);
+    Directory tempDir = getDirectory();
+    FuzzySuggester suggest = new FuzzySuggester(tempDir, "fuzzy", a, a, 0, 256, -1, true, maxEdits, transpositions, prefixLen, prefixLen, false);
 
     if (VERBOSE) {
       System.out.println("TEST: maxEdits=" + maxEdits + " prefixLen=" + prefixLen + " transpositions=" + transpositions + " num=" + NUM);
@@ -1022,7 +1039,7 @@ public class FuzzySuggesterTest extends
       }
       assertEquals(expected.size(), actual.size());
     }
-    a.close();
+    IOUtils.close(a, tempDir);
   }
 
   private List<LookupResult> slowFuzzyMatch(int prefixLen, int maxEdits, boolean allowTransposition, List<Input> answers, String frag) {
@@ -1194,4 +1211,12 @@ public class FuzzySuggesterTest extends
     }
     return ref;
   }
+
+  private Directory getDirectory() {     
+    Directory dir = newDirectory();
+    if (dir instanceof MockDirectoryWrapper) {
+      ((MockDirectoryWrapper) dir).setEnableVirusScanner(false);
+    }
+    return dir;
+  }
 }

Modified: lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/BytesRefSortersTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/BytesRefSortersTest.java?rev=1708760&r1=1708759&r2=1708760&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/BytesRefSortersTest.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/BytesRefSortersTest.java Thu Oct 15 09:58:18 2015
@@ -18,8 +18,11 @@ package org.apache.lucene.search.suggest
  */
 
 import org.apache.lucene.search.suggest.InMemorySorter;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefIterator;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.OfflineSorter;
 import org.junit.Test;
@@ -27,9 +30,13 @@ import org.junit.Test;
 public class BytesRefSortersTest extends LuceneTestCase {
   @Test
   public void testExternalRefSorter() throws Exception {
-    ExternalRefSorter s = new ExternalRefSorter(new OfflineSorter());
+    Directory tempDir = newDirectory();
+    if (tempDir instanceof MockDirectoryWrapper) {
+      ((MockDirectoryWrapper) tempDir).setEnableVirusScanner(false);
+    }
+    ExternalRefSorter s = new ExternalRefSorter(new OfflineSorter(tempDir, "temp"));
     check(s);
-    s.close();
+    IOUtils.close(s, tempDir);
   }
 
   @Test

Modified: lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/FSTCompletionTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/FSTCompletionTest.java?rev=1708760&r1=1708759&r2=1708760&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/FSTCompletionTest.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/FSTCompletionTest.java Thu Oct 15 09:58:18 2015
@@ -20,15 +20,18 @@ package org.apache.lucene.search.suggest
 import java.nio.charset.StandardCharsets;
 import java.util.*;
 
-import org.apache.lucene.search.suggest.Lookup.LookupResult;
 import org.apache.lucene.search.suggest.*;
+import org.apache.lucene.search.suggest.Lookup.LookupResult;
 import org.apache.lucene.search.suggest.fst.FSTCompletion.Completion;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.*;
 
 /**
  * Unit tests for {@link FSTCompletion}.
  */
 public class FSTCompletionTest extends LuceneTestCase {
+
   public static Input tf(String t, int v) {
     return new Input(t, v);
   }
@@ -155,7 +158,8 @@ public class FSTCompletionTest extends L
   }
 
   public void testLargeInputConstantWeights() throws Exception {
-    FSTCompletionLookup lookup = new FSTCompletionLookup(10, true);
+    Directory tempDir = getDirectory();
+    FSTCompletionLookup lookup = new FSTCompletionLookup(tempDir, "fst", 10, true);
     
     Random r = random();
     List<Input> keys = new ArrayList<>();
@@ -175,12 +179,14 @@ public class FSTCompletionTest extends L
       }
       previous = current;
     }
+    tempDir.close();
   }  
 
   public void testMultilingualInput() throws Exception {
     List<Input> input = LookupBenchmarkTest.readTop50KWiki();
 
-    FSTCompletionLookup lookup = new FSTCompletionLookup();
+    Directory tempDir = getDirectory();
+    FSTCompletionLookup lookup = new FSTCompletionLookup(tempDir, "fst");
     lookup.build(new InputArrayIterator(input));
     assertEquals(input.size(), lookup.getCount());
     for (Input tf : input) {
@@ -192,6 +198,7 @@ public class FSTCompletionTest extends L
     assertEquals(5, result.size());
     assertTrue(result.get(0).key.toString().equals("wit"));  // exact match.
     assertTrue(result.get(1).key.toString().equals("with")); // highest count.
+    tempDir.close();
   }
 
   public void testEmptyInput() throws Exception {
@@ -207,7 +214,8 @@ public class FSTCompletionTest extends L
       freqs.add(new Input("" + rnd.nextLong(), weight));
     }
 
-    FSTCompletionLookup lookup = new FSTCompletionLookup();
+    Directory tempDir = getDirectory();
+    FSTCompletionLookup lookup = new FSTCompletionLookup(tempDir, "fst");
     lookup.build(new InputArrayIterator(freqs.toArray(new Input[freqs.size()])));
 
     for (Input tf : freqs) {
@@ -219,6 +227,7 @@ public class FSTCompletionTest extends L
         }
       }
     }
+    tempDir.close();
   }
 
   private CharSequence stringToCharSequence(String prefix) {
@@ -262,4 +271,12 @@ public class FSTCompletionTest extends L
       len = Math.max(len, s.length());
     return len;
   }
+
+  private Directory getDirectory() {     
+    Directory dir = newDirectory();
+    if (dir instanceof MockDirectoryWrapper) {
+      ((MockDirectoryWrapper) dir).setEnableVirusScanner(false);
+    }
+    return dir;
+  }
 }

Modified: lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/WFSTCompletionTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/WFSTCompletionTest.java?rev=1708760&r1=1708759&r2=1708760&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/WFSTCompletionTest.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/WFSTCompletionTest.java Thu Oct 15 09:58:18 2015
@@ -19,9 +19,11 @@ package org.apache.lucene.search.suggest
 
 import java.util.*;
 
-import org.apache.lucene.search.suggest.Lookup.LookupResult;
 import org.apache.lucene.search.suggest.Input;
 import org.apache.lucene.search.suggest.InputArrayIterator;
+import org.apache.lucene.search.suggest.Lookup.LookupResult;
+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;
@@ -37,7 +39,8 @@ public class WFSTCompletionTest extends
     };
     
     Random random = new Random(random().nextLong());
-    WFSTCompletionLookup suggester = new WFSTCompletionLookup();
+    Directory tempDir = getDirectory();
+    WFSTCompletionLookup suggester = new WFSTCompletionLookup(tempDir, "wfst");
     suggester.build(new InputArrayIterator(keys));
     
     // top N of 2, but only foo is available
@@ -75,11 +78,13 @@ public class WFSTCompletionTest extends
     assertEquals(10, results.get(1).value, 0.01F);
     assertEquals("barbara", results.get(2).key.toString());
     assertEquals(6, results.get(2).value, 0.01F);
+    tempDir.close();
   }
 
   public void testExactFirst() throws Exception {
 
-    WFSTCompletionLookup suggester = new WFSTCompletionLookup(true);
+    Directory tempDir = getDirectory();
+    WFSTCompletionLookup suggester = new WFSTCompletionLookup(tempDir, "wfst", true);
 
     suggester.build(new InputArrayIterator(new Input[] {
           new Input("x y", 20),
@@ -99,11 +104,13 @@ public class WFSTCompletionTest extends
         assertEquals(20, results.get(1).value);
       }
     }
+    tempDir.close();
   }
 
   public void testNonExactFirst() throws Exception {
 
-    WFSTCompletionLookup suggester = new WFSTCompletionLookup(false);
+    Directory tempDir = getDirectory();
+    WFSTCompletionLookup suggester = new WFSTCompletionLookup(tempDir, "wfst", false);
 
     suggester.build(new InputArrayIterator(new Input[] {
           new Input("x y", 20),
@@ -123,6 +130,7 @@ public class WFSTCompletionTest extends
         assertEquals(2, results.get(1).value);
       }
     }
+    tempDir.close();
   }
   
   public void testRandom() throws Exception {
@@ -153,7 +161,8 @@ public class WFSTCompletionTest extends
       keys[i] = new Input(s, weight);
     }
 
-    WFSTCompletionLookup suggester = new WFSTCompletionLookup(false);
+    Directory tempDir = getDirectory();
+    WFSTCompletionLookup suggester = new WFSTCompletionLookup(tempDir, "wfst", false);
     suggester.build(new InputArrayIterator(keys));
 
     assertEquals(numWords, suggester.getCount());
@@ -196,6 +205,7 @@ public class WFSTCompletionTest extends
         assertEquals(matches.get(hit).value, r.get(hit).value, 0f);
       }
     }
+    tempDir.close();
   }
 
   public void test0ByteKeys() throws Exception {
@@ -204,20 +214,32 @@ public class WFSTCompletionTest extends
     BytesRef key2 = new BytesRef(3);
     key1.length = 3;
 
-    WFSTCompletionLookup suggester = new WFSTCompletionLookup(false);
+    Directory tempDir = getDirectory();
+    WFSTCompletionLookup suggester = new WFSTCompletionLookup(tempDir, "wfst", false);
 
     suggester.build(new InputArrayIterator(new Input[] {
           new Input(key1, 50),
           new Input(key2, 50),
         }));
+    tempDir.close();
   }
 
   public void testEmpty() throws Exception {
-    WFSTCompletionLookup suggester = new WFSTCompletionLookup(false);
+    Directory tempDir = getDirectory();
+    WFSTCompletionLookup suggester = new WFSTCompletionLookup(tempDir, "wfst", false);
 
     suggester.build(new InputArrayIterator(new Input[0]));
     assertEquals(0, suggester.getCount());
     List<LookupResult> result = suggester.lookup("a", false, 20);
     assertTrue(result.isEmpty());
+    tempDir.close();
+  }
+
+  private Directory getDirectory() {     
+    Directory dir = newDirectory();
+    if (dir instanceof MockDirectoryWrapper) {
+      ((MockDirectoryWrapper) dir).setEnableVirusScanner(false);
+    }
+    return dir;
   }
 }

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java?rev=1708760&r1=1708759&r2=1708760&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java Thu Oct 15 09:58:18 2015
@@ -23,9 +23,12 @@ import java.io.IOException;
 import java.nio.file.Files;
 import java.nio.file.NoSuchFileException;
 import java.nio.file.Path;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.zip.CRC32;
@@ -1165,5 +1168,26 @@ public abstract class BaseDirectoryTestC
     in.close(); // close again
     dir.close();
   }
-}
 
+  public void testCreateTempOutput() throws Throwable {
+    Directory dir = getDirectory(createTempDir());
+    List<String> names = new ArrayList<>();
+    int iters = atLeast(50);
+    for(int iter=0;iter<iters;iter++) {
+      IndexOutput out = dir.createTempOutput("foo", "bar", newIOContext(random()));
+      names.add(out.getName());
+      out.writeVInt(iter);
+      out.close();
+    }
+    for(int iter=0;iter<iters;iter++) {
+      IndexInput in = dir.openInput(names.get(iter), newIOContext(random()));
+      assertEquals(iter, in.readVInt());
+      in.close();
+    }
+    Set<String> files = new HashSet<String>(Arrays.asList(dir.listAll()));
+    // In case ExtraFS struck:
+    files.remove("extra0");
+    assertEquals(new HashSet<String>(names), files);
+    dir.close();
+  }
+}

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java?rev=1708760&r1=1708759&r2=1708760&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java Thu Oct 15 09:58:18 2015
@@ -39,7 +39,7 @@ public class MockIndexOutputWrapper exte
 
   /** Construct an empty output buffer. */
   public MockIndexOutputWrapper(MockDirectoryWrapper dir, IndexOutput delegate, String name) {
-    super("MockIndexOutputWrapper(" + delegate + ")");
+    super("MockIndexOutputWrapper(" + delegate + ")", delegate.getName());
     this.dir = dir;
     this.name = name;
     this.delegate = delegate;

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleTemporaryFilesCleanup.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleTemporaryFilesCleanup.java?rev=1708760&r1=1708759&r2=1708760&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleTemporaryFilesCleanup.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleTemporaryFilesCleanup.java Thu Oct 15 09:58:18 2015
@@ -1,5 +1,22 @@
 package org.apache.lucene.util;
 
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 import java.io.IOException;
 import java.net.URI;
 import java.nio.file.FileSystem;
@@ -30,23 +47,6 @@ import org.apache.lucene.util.LuceneTest
 import com.carrotsearch.randomizedtesting.RandomizedContext;
 import com.carrotsearch.randomizedtesting.rules.TestRuleAdapter;
 
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
 /**
  * Checks and cleans up temporary files.
  * 
@@ -113,9 +113,6 @@ final class TestRuleTemporaryFilesCleanu
     assert tempDirBase == null;
     fileSystem = initializeFileSystem();
     javaTempDir = initializeJavaTempDir();
-
-    // So all code using OfflineSorter (suggesters, BKD tree, NumericRangeTree) see MockFS goodness, e.g. catching leaked file handles:
-    OfflineSorter.setDefaultTempDir(javaTempDir);
   }
   
   // os/config-independent limit for too many open files
@@ -236,7 +233,7 @@ final class TestRuleTemporaryFilesCleanu
     }
   }
   
-  final Path getPerTestClassTempDir() {
+  Path getPerTestClassTempDir() {
     if (tempDirBase == null) {
       RandomizedContext ctx = RandomizedContext.current();
       Class<?> clazz = ctx.getTargetClass();