You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2014/03/07 14:07:45 UTC

svn commit: r1575253 - in /lucene/dev/branches/branch_4x: ./ dev-tools/ lucene/ lucene/misc/ lucene/misc/src/java/org/apache/lucene/index/sorter/ lucene/misc/src/test/org/apache/lucene/index/sorter/ lucene/suggest/ lucene/suggest/src/java/org/apache/lu...

Author: rmuir
Date: Fri Mar  7 13:07:45 2014
New Revision: 1575253

URL: http://svn.apache.org/r1575253
Log:
LUCENE-5493: cut over index sorting to use Sort api for specifying the order

Added:
    lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/index/sorter/BlockJoinComparatorSource.java
      - copied unchanged from r1575248, lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/index/sorter/BlockJoinComparatorSource.java
Removed:
    lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/index/sorter/BlockJoinSorter.java
    lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/index/sorter/NumericDocValuesSorter.java
Modified:
    lucene/dev/branches/branch_4x/   (props changed)
    lucene/dev/branches/branch_4x/dev-tools/   (props changed)
    lucene/dev/branches/branch_4x/lucene/   (props changed)
    lucene/dev/branches/branch_4x/lucene/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_4x/lucene/misc/   (props changed)
    lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/index/sorter/EarlyTerminatingSortingCollector.java
    lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/index/sorter/Sorter.java
    lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/index/sorter/SortingAtomicReader.java
    lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/index/sorter/SortingMergePolicy.java
    lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/index/sorter/package.html
    lucene/dev/branches/branch_4x/lucene/misc/src/test/org/apache/lucene/index/sorter/IndexSortingTest.java
    lucene/dev/branches/branch_4x/lucene/misc/src/test/org/apache/lucene/index/sorter/SortingAtomicReaderTest.java
    lucene/dev/branches/branch_4x/lucene/misc/src/test/org/apache/lucene/index/sorter/TestBlockJoinSorter.java
    lucene/dev/branches/branch_4x/lucene/misc/src/test/org/apache/lucene/index/sorter/TestEarlyTermination.java
    lucene/dev/branches/branch_4x/lucene/misc/src/test/org/apache/lucene/index/sorter/TestSortingMergePolicy.java
    lucene/dev/branches/branch_4x/lucene/suggest/   (props changed)
    lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
    lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggester.java
    lucene/dev/branches/branch_4x/lucene/suggest/src/test/org/apache/lucene/search/suggest/LookupBenchmarkTest.java
    lucene/dev/branches/branch_4x/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java
    lucene/dev/branches/branch_4x/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggesterTest.java
    lucene/dev/branches/branch_4x/lucene/test-framework/   (props changed)
    lucene/dev/branches/branch_4x/solr/   (props changed)
    lucene/dev/branches/branch_4x/solr/core/   (props changed)
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/spelling/suggest/fst/AnalyzingInfixLookupFactory.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/spelling/suggest/fst/BlendedInfixLookupFactory.java

Modified: lucene/dev/branches/branch_4x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/CHANGES.txt?rev=1575253&r1=1575252&r2=1575253&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_4x/lucene/CHANGES.txt Fri Mar  7 13:07:45 2014
@@ -36,6 +36,10 @@ New Features
 * LUCENE-5224: Add iconv, oconv, and ignore support to HunspellStemFilter.
   (Robert Muir)
 
+* LUCENE-5493: SortingMergePolicy, and EarlyTerminatingSortingCollector
+  support arbitrary Sort specifications.  
+  (Robert Muir, Mike McCandless, Adrien Grand)
+
 API Changes
 
 * LUCENE-5454: Add RandomAccessOrds, an optional extension of SortedSetDocValues
@@ -43,6 +47,12 @@ API Changes
 
 * LUCENE-5468: Move offline Sort (from suggest module) to OfflineSort. (Robert Muir)
 
+* LUCENE-5493: SortingMergePolicy and EarlyTerminatingSortingCollector take
+  Sort instead of Sorter. BlockJoinSorter is removed, replaced with 
+  BlockJoinComparatorSource, which can take a Sort for ordering of parents
+  and a separate Sort for ordering of children within a block. 
+  (Robert Muir, Mike McCandless, Adrien Grand)
+
 Optimizations
 
 * LUCENE-5468: HunspellStemFilter uses 10 to 100x less RAM. It also loads

Modified: lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/index/sorter/EarlyTerminatingSortingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/index/sorter/EarlyTerminatingSortingCollector.java?rev=1575253&r1=1575252&r2=1575253&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/index/sorter/EarlyTerminatingSortingCollector.java (original)
+++ lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/index/sorter/EarlyTerminatingSortingCollector.java Fri Mar  7 13:07:45 2014
@@ -24,50 +24,53 @@ import org.apache.lucene.index.IndexWrit
 import org.apache.lucene.search.CollectionTerminatedException;
 import org.apache.lucene.search.Collector;
 import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.TopDocsCollector;
 import org.apache.lucene.search.TotalHitCountCollector;
 
 /**
  * A {@link Collector} that early terminates collection of documents on a
  * per-segment basis, if the segment was sorted according to the given
- * {@link Sorter}.
+ * {@link Sort}.
  * 
  * <p>
- * <b>NOTE:</b> the {@link Collector} detects sorted segments according to
+ * <b>NOTE:</b> the {@code Collector} detects sorted segments according to
  * {@link SortingMergePolicy}, so it's best used in conjunction with it. Also,
- * it collects up to a specified num docs from each segment, and therefore is
- * mostly suitable for use in conjunction with collectors such as
+ * it collects up to a specified {@code numDocsToCollect} from each segment, 
+ * and therefore is mostly suitable for use in conjunction with collectors such as
  * {@link TopDocsCollector}, and not e.g. {@link TotalHitCountCollector}.
  * <p>
- * <b>NOTE</b>: If you wrap a {@link TopDocsCollector} that sorts in the same
- * order as the index order, the returned {@link TopDocsCollector#topDocs()}
+ * <b>NOTE</b>: If you wrap a {@code TopDocsCollector} that sorts in the same
+ * order as the index order, the returned {@link TopDocsCollector#topDocs() TopDocs}
  * will be correct. However the total of {@link TopDocsCollector#getTotalHits()
  * hit count} will be underestimated since not all matching documents will have
  * been collected.
  * <p>
- * <b>NOTE</b>: This {@link Collector} uses {@link Sorter#getID()} to detect
- * whether a segment was sorted with the same {@link Sorter} as the one given in
- * {@link #EarlyTerminatingSortingCollector(Collector, Sorter, int)}. This has
+ * <b>NOTE</b>: This {@code Collector} uses {@link Sort#toString()} to detect
+ * whether a segment was sorted with the same {@code Sort}. This has
  * two implications:
  * <ul>
- * <li>if {@link Sorter#getID()} is not implemented correctly and returns
- * different identifiers for equivalent {@link Sorter}s, this collector will not
+ * <li>if a custom comparator is not implemented correctly and returns
+ * different identifiers for equivalent instances, this collector will not
  * detect sorted segments,</li>
  * <li>if you suddenly change the {@link IndexWriter}'s
- * {@link SortingMergePolicy} to sort according to another criterion and if both
- * the old and the new {@link Sorter}s have the same identifier, this
- * {@link Collector} will incorrectly detect sorted segments.</li>
+ * {@code SortingMergePolicy} to sort according to another criterion and if both
+ * the old and the new {@code Sort}s have the same identifier, this
+ * {@code Collector} will incorrectly detect sorted segments.</li>
  * </ul>
  * 
  * @lucene.experimental
  */
 public class EarlyTerminatingSortingCollector extends Collector {
-
+  /** The wrapped Collector */
   protected final Collector in;
-  protected final Sorter sorter;
+  /** Sort used to sort the search results */
+  protected final Sort sort;
+  /** Number of documents to collect in each segment */
   protected final int numDocsToCollect;
-  
+  /** Number of documents to collect in the current segment being processed */
   protected int segmentTotalCollect;
+  /** True if the current segment being processed is sorted by {@link #sort} */
   protected boolean segmentSorted;
 
   private int numCollected;
@@ -77,20 +80,19 @@ public class EarlyTerminatingSortingColl
    * 
    * @param in
    *          the collector to wrap
-   * @param sorter
-   *          the same sorter as the one which is used by {@link IndexWriter}'s
-   *          {@link SortingMergePolicy}
+   * @param sort
+   *          the sort you are sorting the search results on
    * @param numDocsToCollect
    *          the number of documents to collect on each segment. When wrapping
    *          a {@link TopDocsCollector}, this number should be the number of
    *          hits.
    */
-  public EarlyTerminatingSortingCollector(Collector in, Sorter sorter, int numDocsToCollect) {
+  public EarlyTerminatingSortingCollector(Collector in, Sort sort, int numDocsToCollect) {
     if (numDocsToCollect <= 0) {
       throw new IllegalStateException("numDocsToCollect must always be > 0, got " + segmentTotalCollect);
     }
     this.in = in;
-    this.sorter = sorter;
+    this.sort = sort;
     this.numDocsToCollect = numDocsToCollect;
   }
 
@@ -110,7 +112,7 @@ public class EarlyTerminatingSortingColl
   @Override
   public void setNextReader(AtomicReaderContext context) throws IOException {
     in.setNextReader(context);
-    segmentSorted = SortingMergePolicy.isSorted(context.reader(), sorter);
+    segmentSorted = SortingMergePolicy.isSorted(context.reader(), sort);
     segmentTotalCollect = segmentSorted ? numDocsToCollect : Integer.MAX_VALUE;
     numCollected = 0;
   }

Modified: lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/index/sorter/Sorter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/index/sorter/Sorter.java?rev=1575253&r1=1575252&r2=1575253&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/index/sorter/Sorter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/index/sorter/Sorter.java Fri Mar  7 13:07:45 2014
@@ -22,47 +22,44 @@ import java.util.Comparator;
 
 import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.FieldComparator;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
 import org.apache.lucene.util.TimSorter;
 import org.apache.lucene.util.packed.MonotonicAppendingLongBuffer;
 
 /**
  * Sorts documents of a given index by returning a permutation on the document
  * IDs.
- * <p><b>NOTE</b>: A {@link Sorter} implementation can be easily written from
- * a {@link DocComparator document comparator} by using the
- * {@link #sort(int, DocComparator)} helper method. This is especially useful
- * when documents are directly comparable by their field values.
  * @lucene.experimental
  */
-public abstract class Sorter {
-
-  /** A comparator that keeps documents in index order. */
-  public static final DocComparator INDEX_ORDER_COMPARATOR = new DocComparator() {
-    @Override
-    public int compare(int docID1, int docID2) {
-      return docID1 - docID2;
-    }
-  };
+final class Sorter {
+  final Sort sort;
+  
+  /** Creates a new Sorter to sort the index with {@code sort} */
+  Sorter(Sort sort) {
+    this.sort = sort;
+  }
 
   /**
    * A permutation of doc IDs. For every document ID between <tt>0</tt> and
    * {@link IndexReader#maxDoc()}, <code>oldToNew(newToOld(docID))</code> must
    * return <code>docID</code>.
    */
-  public static abstract class DocMap {
+  static abstract class DocMap {
 
     /** Given a doc ID from the original index, return its ordinal in the
      *  sorted index. */
-    public abstract int oldToNew(int docID);
+    abstract int oldToNew(int docID);
 
     /** Given the ordinal of a doc ID, return its doc ID in the original index. */
-    public abstract int newToOld(int docID);
+    abstract int newToOld(int docID);
 
     /** Return the number of documents in this map. This must be equal to the
      *  {@link AtomicReader#maxDoc() number of documents} of the
      *  {@link AtomicReader} which is sorted. */
-    public abstract int size();
-
+    abstract int size();
   }
 
   /** Check consistency of a {@link DocMap}, useful for assertions. */
@@ -81,7 +78,7 @@ public abstract class Sorter {
   }
 
   /** A comparator of doc IDs. */
-  public static abstract class DocComparator {
+  static abstract class DocComparator {
 
     /** Compare docID1 against docID2. The contract for the return value is the
      *  same as {@link Comparator#compare(Object, Object)}. */
@@ -89,45 +86,13 @@ public abstract class Sorter {
 
   }
 
-  /**
-   * Sorts documents in reverse order. <b>NOTE</b>: This {@link Sorter} is not
-   * idempotent. Sorting an {@link AtomicReader} once or twice will return two
-   * different {@link AtomicReader} views. This {@link Sorter} should not be
-   * used with {@link SortingMergePolicy}.
-   */
-  public static final Sorter REVERSE_DOCS = new Sorter() {
-    @Override
-    public DocMap sort(final AtomicReader reader) throws IOException {
-      final int maxDoc = reader.maxDoc();
-      return new DocMap() {
-        @Override
-        public int oldToNew(int docID) {
-          return maxDoc - docID - 1;
-        }
-        @Override
-        public int newToOld(int docID) {
-          return maxDoc - docID - 1;
-        }
-        @Override
-        public int size() {
-          return maxDoc;
-        }
-      };
-    }
-    
-    @Override
-    public String getID() {
-      return "ReverseDocs";
-    }
-  };
-  
   private static final class DocValueSorter extends TimSorter {
     
     private final int[] docs;
     private final Sorter.DocComparator comparator;
     private final int[] tmp;
     
-    public DocValueSorter(int[] docs, Sorter.DocComparator comparator) {
+    DocValueSorter(int[] docs, Sorter.DocComparator comparator) {
       super(docs.length / 64);
       this.docs = docs;
       this.comparator = comparator;
@@ -168,7 +133,7 @@ public abstract class Sorter {
   }
 
   /** Computes the old-to-new permutation over the given comparator. */
-  protected static Sorter.DocMap sort(final int maxDoc, DocComparator comparator) {
+  private static Sorter.DocMap sort(final int maxDoc, DocComparator comparator) {
     // check if the index is sorted
     boolean sorted = true;
     for (int i = 1; i < maxDoc; ++i) {
@@ -242,20 +207,75 @@ public abstract class Sorter {
    * <b>NOTE:</b> deleted documents are expected to appear in the mapping as
    * well, they will however be marked as deleted in the sorted view.
    */
-  public abstract DocMap sort(AtomicReader reader) throws IOException;
+  DocMap sort(AtomicReader reader) throws IOException {
+    SortField fields[] = sort.getSort();
+    final int reverseMul[] = new int[fields.length];
+    final FieldComparator<?> comparators[] = new FieldComparator[fields.length];
+    
+    for (int i = 0; i < fields.length; i++) {
+      reverseMul[i] = fields[i].getReverse() ? -1 : 1;
+      comparators[i] = fields[i].getComparator(1, i);
+      comparators[i].setNextReader(reader.getContext());
+      comparators[i].setScorer(FAKESCORER);
+    }
+    final DocComparator comparator = new DocComparator() {
+      @Override
+      public int compare(int docID1, int docID2) {
+        try {
+          for (int i = 0; i < comparators.length; i++) {
+            // TODO: would be better if copy() didnt cause a term lookup in TermOrdVal & co,
+            // the segments are always the same here...
+            comparators[i].copy(0, docID1);
+            comparators[i].setBottom(0);
+            int comp = reverseMul[i] * comparators[i].compareBottom(docID2);
+            if (comp != 0) {
+              return comp;
+            }
+          }
+          return Integer.compare(docID1, docID2); // docid order tiebreak
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+      }
+    };
+    return sort(reader.maxDoc(), comparator);
+  }
 
   /**
    * Returns the identifier of this {@link Sorter}.
    * <p>This identifier is similar to {@link Object#hashCode()} and should be
    * chosen so that two instances of this class that sort documents likewise
    * will have the same identifier. On the contrary, this identifier should be
-   * different on different {@link Sorter sorters}.
+   * different on different {@link Sort sorts}.
    */
-  public abstract String getID();
+  public String getID() {
+    return sort.toString();
+  }
 
   @Override
   public String toString() {
     return getID();
   }
   
+  static final Scorer FAKESCORER = new Scorer(null) {
+    
+    @Override
+    public float score() throws IOException { throw new UnsupportedOperationException(); }
+    
+    @Override
+    public int freq() throws IOException { throw new UnsupportedOperationException(); }
+
+    @Override
+    public int docID() { throw new UnsupportedOperationException(); }
+
+    @Override
+    public int nextDoc() throws IOException { throw new UnsupportedOperationException(); }
+
+    @Override
+    public int advance(int target) throws IOException { throw new UnsupportedOperationException(); }
+
+    @Override
+    public long cost() { throw new UnsupportedOperationException(); }
+  };
+  
 }

Modified: lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/index/sorter/SortingAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/index/sorter/SortingAtomicReader.java?rev=1575253&r1=1575252&r2=1575253&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/index/sorter/SortingAtomicReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/index/sorter/SortingAtomicReader.java Fri Mar  7 13:07:45 2014
@@ -35,6 +35,7 @@ import org.apache.lucene.index.StoredFie
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.Sort;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.RAMFile;
@@ -48,13 +49,13 @@ import org.apache.lucene.util.automaton.
 
 /**
  * An {@link AtomicReader} which supports sorting documents by a given
- * {@link Sorter}. You can use this class to sort an index as follows:
+ * {@link Sort}. You can use this class to sort an index as follows:
  * 
  * <pre class="prettyprint">
  * IndexWriter writer; // writer to which the sorted index will be added
  * DirectoryReader reader; // reader on the input index
- * Sorter sorter; // determines how the documents are sorted
- * AtomicReader sortingReader = SortingAtomicReader.wrap(SlowCompositeReaderWrapper.wrap(reader), sorter);
+ * Sort sort; // determines how the documents are sorted
+ * AtomicReader sortingReader = SortingAtomicReader.wrap(SlowCompositeReaderWrapper.wrap(reader), sort);
  * writer.addIndexes(reader);
  * writer.close();
  * reader.close();
@@ -480,7 +481,7 @@ public class SortingAtomicReader extends
   static class SortingDocsAndPositionsEnum extends FilterDocsAndPositionsEnum {
     
     /**
-     * A {@link Sorter} which sorts two parallel arrays of doc IDs and
+     * A {@link TimSorter} which sorts two parallel arrays of doc IDs and
      * offsets in one go. Everytime a doc ID is 'swapped', its correponding offset
      * is swapped too.
      */
@@ -708,14 +709,14 @@ public class SortingAtomicReader extends
   }
 
   /** Return a sorted view of <code>reader</code> according to the order
-   *  defined by <code>sorter</code>. If the reader is already sorted, this
+   *  defined by <code>sort</code>. If the reader is already sorted, this
    *  method might return the reader as-is. */
-  public static AtomicReader wrap(AtomicReader reader, Sorter sorter) throws IOException {
-    return wrap(reader, sorter.sort(reader));
+  public static AtomicReader wrap(AtomicReader reader, Sort sort) throws IOException {
+    return wrap(reader, new Sorter(sort).sort(reader));
   }
 
-  /** Expert: same as {@link #wrap(AtomicReader, Sorter)} but operates directly on a {@link Sorter.DocMap}. */
-  public static AtomicReader wrap(AtomicReader reader, Sorter.DocMap docMap) {
+  /** Expert: same as {@link #wrap(AtomicReader, Sort)} but operates directly on a {@link Sorter.DocMap}. */
+  static AtomicReader wrap(AtomicReader reader, Sorter.DocMap docMap) {
     if (docMap == null) {
       // the reader is already sorter
       return reader;

Modified: lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/index/sorter/SortingMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/index/sorter/SortingMergePolicy.java?rev=1575253&r1=1575252&r2=1575253&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/index/sorter/SortingMergePolicy.java (original)
+++ lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/index/sorter/SortingMergePolicy.java Fri Mar  7 13:07:45 2014
@@ -22,6 +22,7 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.lucene.analysis.Analyzer; // javadocs
 import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
@@ -33,22 +34,23 @@ import org.apache.lucene.index.SegmentCo
 import org.apache.lucene.index.SegmentInfos;
 import org.apache.lucene.index.SegmentReader;
 import org.apache.lucene.index.SlowCompositeReaderWrapper;
+import org.apache.lucene.search.Sort;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.packed.MonotonicAppendingLongBuffer;
 
-/** A {@link MergePolicy} that reorders documents according to a {@link Sorter}
+/** A {@link MergePolicy} that reorders documents according to a {@link Sort}
  *  before merging them. As a consequence, all segments resulting from a merge
  *  will be sorted while segments resulting from a flush will be in the order
  *  in which documents have been added.
- *  <p><b>NOTE</b>: Never use this {@link MergePolicy} if you rely on
- *  {@link IndexWriter#addDocuments(Iterable, org.apache.lucene.analysis.Analyzer)}
+ *  <p><b>NOTE</b>: Never use this policy if you rely on
+ *  {@link IndexWriter#addDocuments(Iterable, Analyzer) IndexWriter.addDocuments}
  *  to have sequentially-assigned doc IDs, this policy will scatter doc IDs.
- *  <p><b>NOTE</b>: This {@link MergePolicy} should only be used with idempotent
- *  {@link Sorter}s so that the order of segments is predictable. For example,
- *  using {@link SortingMergePolicy} with {@link Sorter#REVERSE_DOCS} (which is
- *  not idempotent) will make the order of documents in a segment depend on the
- *  number of times the segment has been merged.
+ *  <p><b>NOTE</b>: This policy should only be used with idempotent {@code Sort}s 
+ *  so that the order of segments is predictable. For example, using 
+ *  {@link Sort#INDEXORDER} in reverse (which is not idempotent) will make 
+ *  the order of documents in a segment depend on the number of times the segment 
+ *  has been merged.
  *  @lucene.experimental */
 public final class SortingMergePolicy extends MergePolicy {
 
@@ -147,12 +149,12 @@ public final class SortingMergePolicy ex
 
   }
 
-  /** Returns true if the given reader is sorted by the given sorter. */
-  public static boolean isSorted(AtomicReader reader, Sorter sorter) {
+  /** Returns {@code true} if the given {@code reader} is sorted by the specified {@code sort}. */
+  public static boolean isSorted(AtomicReader reader, Sort sort) {
     if (reader instanceof SegmentReader) {
       final SegmentReader segReader = (SegmentReader) reader;
       final Map<String, String> diagnostics = segReader.getSegmentInfo().info.getDiagnostics();
-      if (diagnostics != null && sorter.getID().equals(diagnostics.get(SORTER_ID_PROP))) {
+      if (diagnostics != null && sort.toString().equals(diagnostics.get(SORTER_ID_PROP))) {
         return true;
       }
     }
@@ -172,11 +174,13 @@ public final class SortingMergePolicy ex
 
   final MergePolicy in;
   final Sorter sorter;
+  final Sort sort;
 
-  /** Create a new {@link MergePolicy} that sorts documents with <code>sorter</code>. */
-  public SortingMergePolicy(MergePolicy in, Sorter sorter) {
+  /** Create a new {@code MergePolicy} that sorts documents with the given {@code sort}. */
+  public SortingMergePolicy(MergePolicy in, Sort sort) {
     this.in = in;
-    this.sorter = sorter;
+    this.sorter = new Sorter(sort);
+    this.sort = sort;
   }
 
   @Override
@@ -200,7 +204,7 @@ public final class SortingMergePolicy ex
 
   @Override
   public MergePolicy clone() {
-    return new SortingMergePolicy(in.clone(), sorter);
+    return new SortingMergePolicy(in.clone(), sort);
   }
 
   @Override

Modified: lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/index/sorter/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/index/sorter/package.html?rev=1575253&r1=1575252&r2=1575253&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/index/sorter/package.html (original)
+++ lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/index/sorter/package.html Fri Mar  7 13:07:45 2014
@@ -17,19 +17,16 @@
 -->
 <html>
 <body>
-<p>Provides index sorting capablities. The application can use one of the
-pre-existing Sorter implementations, e.g. to sort by a
-{@link org.apache.lucene.index.sorter.NumericDocValuesSorter}
-or {@link org.apache.lucene.index.sorter.Sorter#REVERSE_DOCS reverse} the order
-of the documents. Additionally, the application can implement a custom
-{@link org.apache.lucene.index.sorter.Sorter} which returns a permutation on 
-a source {@link org.apache.lucene.index.AtomicReader}'s document IDs, to sort
-the input documents by additional criteria.
+<p>Provides index sorting capablities. The application can use any
+Sort specification, e.g. to sort by fields using DocValues or FieldCache, or to
+reverse the order of the documents (by using SortField.Type.DOC in reverse).
+Multi-level sorts can be specified the same way you would when searching, by
+building Sort from multiple SortFields.
 
 <p>{@link org.apache.lucene.index.sorter.SortingMergePolicy} can be used to
 make Lucene sort segments before merging them. This will ensure that every
 segment resulting from a merge will be sorted according to the provided
-{@link org.apache.lucene.index.sorter.Sorter}. This however makes merging and
+{@link org.apache.lucene.search.Sort}. This however makes merging and
 thus indexing slower.
 
 <p>Sorted segments allow for early query termination when the sort order

Modified: lucene/dev/branches/branch_4x/lucene/misc/src/test/org/apache/lucene/index/sorter/IndexSortingTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/misc/src/test/org/apache/lucene/index/sorter/IndexSortingTest.java?rev=1575253&r1=1575252&r2=1575253&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/misc/src/test/org/apache/lucene/index/sorter/IndexSortingTest.java (original)
+++ lucene/dev/branches/branch_4x/lucene/misc/src/test/org/apache/lucene/index/sorter/IndexSortingTest.java Fri Mar  7 13:07:45 2014
@@ -24,6 +24,8 @@ import java.util.List;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.SlowCompositeReaderWrapper;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.TestUtil;
@@ -31,9 +33,9 @@ import org.junit.BeforeClass;
 
 public class IndexSortingTest extends SorterTestBase {
   
-  private static final Sorter[] SORTERS = new Sorter[] {
-    new NumericDocValuesSorter(NUMERIC_DV_FIELD, true),
-    Sorter.REVERSE_DOCS,
+  private static final Sort[] SORT = new Sort[] {
+    new Sort(new SortField(NUMERIC_DV_FIELD, SortField.Type.LONG)),
+    new Sort(new SortField(null, SortField.Type.DOC, true))
   };
   
   @BeforeClass
@@ -47,13 +49,14 @@ public class IndexSortingTest extends So
         values.add(Integer.valueOf(reader.document(i).get(ID_FIELD)));
       }
     }
-    Sorter sorter = SORTERS[random().nextInt(SORTERS.length)];
-    if (sorter == Sorter.REVERSE_DOCS) {
+    int idx = random().nextInt(SORT.length);
+    Sort sorter = SORT[idx];
+    if (idx == 1) { // reverse doc sort
       Collections.reverse(values);
     } else {
       Collections.sort(values);
-      if (sorter instanceof NumericDocValuesSorter && random().nextBoolean()) {
-        sorter = new NumericDocValuesSorter(NUMERIC_DV_FIELD, false); // descending
+      if (random().nextBoolean()) {
+        sorter = new Sort(new SortField(NUMERIC_DV_FIELD, SortField.Type.LONG, true)); // descending
         Collections.reverse(values);
       }
     }

Modified: lucene/dev/branches/branch_4x/lucene/misc/src/test/org/apache/lucene/index/sorter/SortingAtomicReaderTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/misc/src/test/org/apache/lucene/index/sorter/SortingAtomicReaderTest.java?rev=1575253&r1=1575252&r2=1575253&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/misc/src/test/org/apache/lucene/index/sorter/SortingAtomicReaderTest.java (original)
+++ lucene/dev/branches/branch_4x/lucene/misc/src/test/org/apache/lucene/index/sorter/SortingAtomicReaderTest.java Fri Mar  7 13:07:45 2014
@@ -17,56 +17,37 @@ package org.apache.lucene.index.sorter;
  * limitations under the License.
  */
 
-import java.io.IOException;
 import java.util.Arrays;
 
-import org.apache.lucene.index.AtomicReader;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.TestUtil;
-import org.apache.lucene.util.TestUtil;
 import org.junit.BeforeClass;
 
 public class SortingAtomicReaderTest extends SorterTestBase {
   
   @BeforeClass
   public static void beforeClassSortingAtomicReaderTest() throws Exception {
-    // build the mapping from the reader, since we deleted documents, some of
-    // them might have disappeared from the index (e.g. if an entire segment is
-    // dropped b/c all its docs are deleted)
-    final int[] values = new int[reader.maxDoc()];
-    for (int i = 0; i < reader.maxDoc(); i++) {
-      values[i] = Integer.valueOf(reader.document(i).get(ID_FIELD));
-    }
-    final Sorter.DocComparator comparator = new Sorter.DocComparator() {
-      @Override
-      public int compare(int docID1, int docID2) {
-        final int v1 = values[docID1];
-        final int v2 = values[docID2];
-        return v1 < v2 ? -1 : v1 == v2 ? 0 : 1;
-      }
-    };
-
-    final Sorter.DocMap docMap = Sorter.sort(reader.maxDoc(), comparator);
+    
+    // sort the index by id (as integer, in NUMERIC_DV_FIELD)
+    Sort sort = new Sort(new SortField(NUMERIC_DV_FIELD, SortField.Type.INT));
+    final Sorter.DocMap docMap = new Sorter(sort).sort(reader);
+ 
     // Sorter.compute also sorts the values
+    NumericDocValues dv = reader.getNumericDocValues(NUMERIC_DV_FIELD);
     sortedValues = new Integer[reader.maxDoc()];
     for (int i = 0; i < reader.maxDoc(); ++i) {
-      sortedValues[docMap.oldToNew(i)] = values[i];
+      sortedValues[docMap.oldToNew(i)] = (int)dv.get(i);
     }
     if (VERBOSE) {
       System.out.println("docMap: " + docMap);
       System.out.println("sortedValues: " + Arrays.toString(sortedValues));
     }
     
-    reader = SortingAtomicReader.wrap(reader, new Sorter() {
-      @Override
-      public Sorter.DocMap sort(AtomicReader reader) throws IOException {
-        return docMap;
-      }
-      @Override
-      public String getID() {
-        return ID_FIELD;
-      }
-    });
+    // sort the index by id (as integer, in NUMERIC_DV_FIELD)
+    reader = SortingAtomicReader.wrap(reader, sort);
     
     if (VERBOSE) {
       System.out.print("mapped-deleted-docs: ");

Modified: lucene/dev/branches/branch_4x/lucene/misc/src/test/org/apache/lucene/index/sorter/TestBlockJoinSorter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/misc/src/test/org/apache/lucene/index/sorter/TestBlockJoinSorter.java?rev=1575253&r1=1575252&r2=1575253&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/misc/src/test/org/apache/lucene/index/sorter/TestBlockJoinSorter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/misc/src/test/org/apache/lucene/index/sorter/TestBlockJoinSorter.java Fri Mar  7 13:07:45 2014
@@ -37,6 +37,8 @@ import org.apache.lucene.search.DocIdSet
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.Filter;
 import org.apache.lucene.search.QueryWrapperFilter;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.FixedBitSet;
@@ -91,51 +93,15 @@ public class TestBlockJoinSorter extends
     final AtomicReader reader = getOnlySegmentReader(indexReader);
     final Filter parentsFilter = new FixedBitSetCachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("parent", "true"))));
     final FixedBitSet parentBits = (FixedBitSet) parentsFilter.getDocIdSet(reader.getContext(), null);
-
     final NumericDocValues parentValues = reader.getNumericDocValues("parent_val");
-    final Sorter.DocComparator parentComparator = new Sorter.DocComparator() {
-      @Override
-      public int compare(int docID1, int docID2) {
-        assertTrue(parentBits.get(docID1));
-        assertTrue(parentBits.get(docID2));
-        final long v1 = parentValues.get(docID1);
-        final long v2 = parentValues.get(docID2);
-        return v1 < v2 ? -1 : v1 == v2 ? 0 : 1;
-      }
-    };
 
     final NumericDocValues childValues = reader.getNumericDocValues("child_val");
-    final Sorter.DocComparator childComparator = new Sorter.DocComparator() {
-      @Override
-      public int compare(int docID1, int docID2) {
-        assertFalse(parentBits.get(docID1));
-        assertFalse(parentBits.get(docID2));
-        final long v1 = childValues.get(docID1);
-        final long v2 = childValues.get(docID2);
-        return v1 < v2 ? -1 : v1 == v2 ? 0 : 1;
-      }
-    };
 
-    final Sorter sorter = new BlockJoinSorter(parentsFilter) {
-      
-      @Override
-      public String getID() {
-        return "Dummy";
-      }
-      
-      @Override
-      protected DocComparator getParentComparator(AtomicReader r) {
-        assertEquals(reader, r);
-        return parentComparator;
-      }
-
-      @Override
-      protected DocComparator getChildComparator(AtomicReader r) {
-        assertEquals(reader, r);
-        return childComparator;
-      }
+    final Sort parentSort = new Sort(new SortField("parent_val", SortField.Type.LONG));
+    final Sort childSort = new Sort(new SortField("child_val", SortField.Type.LONG));
 
-    };
+    final Sort sort = new Sort(new SortField("custom", new BlockJoinComparatorSource(parentsFilter, parentSort, childSort)));
+    final Sorter sorter = new Sorter(sort);
     final Sorter.DocMap docMap = sorter.sort(reader);
     assertEquals(reader.maxDoc(), docMap.size());
 

Modified: lucene/dev/branches/branch_4x/lucene/misc/src/test/org/apache/lucene/index/sorter/TestEarlyTermination.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/misc/src/test/org/apache/lucene/index/sorter/TestEarlyTermination.java?rev=1575253&r1=1575252&r2=1575253&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/misc/src/test/org/apache/lucene/index/sorter/TestEarlyTermination.java (original)
+++ lucene/dev/branches/branch_4x/lucene/misc/src/test/org/apache/lucene/index/sorter/TestEarlyTermination.java Fri Mar  7 13:07:45 2014
@@ -53,14 +53,14 @@ public class TestEarlyTermination extend
   private int numDocs;
   private List<String> terms;
   private Directory dir;
-  private Sorter sorter;
+  private Sort sort;
   private RandomIndexWriter iw;
   private IndexReader reader;
 
   @Override
   public void setUp() throws Exception {
     super.setUp();
-    sorter = new NumericDocValuesSorter("ndv1");
+    sort = new Sort(new SortField("ndv1", SortField.Type.LONG));
   }
 
   private Document randomDocument() {
@@ -82,7 +82,7 @@ public class TestEarlyTermination extend
     terms = new ArrayList<String>(randomTerms);
     final long seed = random().nextLong();
     final IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(new Random(seed)));
-    iwc.setMergePolicy(TestSortingMergePolicy.newSortingMergePolicy(sorter));
+    iwc.setMergePolicy(TestSortingMergePolicy.newSortingMergePolicy(sort));
     iw = new RandomIndexWriter(new Random(seed), dir, iwc);
     for (int i = 0; i < numDocs; ++i) {
       final Document doc = randomDocument();
@@ -122,7 +122,7 @@ public class TestEarlyTermination extend
     for (int i = 0; i < iters; ++i) {
       final TermQuery query = new TermQuery(new Term("s", RandomPicks.randomFrom(random(), terms)));
       searcher.search(query, collector1);
-      searcher.search(query, new EarlyTerminatingSortingCollector(collector2, sorter, numHits));
+      searcher.search(query, new EarlyTerminatingSortingCollector(collector2, sort, numHits));
     }
     assertTrue(collector1.getTotalHits() >= collector2.getTotalHits());
     assertTopDocsEquals(collector1.topDocs().scoreDocs, collector2.topDocs().scoreDocs);
@@ -146,7 +146,8 @@ public class TestEarlyTermination extend
     for (int i = 0; i < iters; ++i) {
       final TermQuery query = new TermQuery(new Term("s", RandomPicks.randomFrom(random(), terms)));
       searcher.search(query, collector1);
-      searcher.search(query, new EarlyTerminatingSortingCollector(collector2, new NumericDocValuesSorter("ndv2"), numHits) {
+      Sort different = new Sort(new SortField("ndv2", SortField.Type.LONG));
+      searcher.search(query, new EarlyTerminatingSortingCollector(collector2, different, numHits) {
         @Override
         public void setNextReader(AtomicReaderContext context) throws IOException {
           super.setNextReader(context);

Modified: lucene/dev/branches/branch_4x/lucene/misc/src/test/org/apache/lucene/index/sorter/TestSortingMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/misc/src/test/org/apache/lucene/index/sorter/TestSortingMergePolicy.java?rev=1575253&r1=1575252&r2=1575253&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/misc/src/test/org/apache/lucene/index/sorter/TestSortingMergePolicy.java (original)
+++ lucene/dev/branches/branch_4x/lucene/misc/src/test/org/apache/lucene/index/sorter/TestSortingMergePolicy.java Fri Mar  7 13:07:45 2014
@@ -40,6 +40,8 @@ import org.apache.lucene.index.RandomInd
 import org.apache.lucene.index.SlowCompositeReaderWrapper;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TieredMergePolicy;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
@@ -52,14 +54,14 @@ public class TestSortingMergePolicy exte
 
   private List<String> terms;
   private Directory dir1, dir2;
-  private Sorter sorter;
+  private Sort sort;
   private IndexReader reader;
   private IndexReader sortedReader;
 
   @Override
   public void setUp() throws Exception {
     super.setUp();
-    sorter = new NumericDocValuesSorter("ndv");
+    sort = new Sort(new SortField("ndv", SortField.Type.LONG));
     createRandomIndexes();
   }
 
@@ -70,7 +72,7 @@ public class TestSortingMergePolicy exte
     return doc;
   }
 
-  static MergePolicy newSortingMergePolicy(Sorter sorter) {
+  static MergePolicy newSortingMergePolicy(Sort sort) {
     // create a MP with a low merge factor so that many merges happen
     MergePolicy mp;
     if (random().nextBoolean()) {
@@ -85,7 +87,7 @@ public class TestSortingMergePolicy exte
       mp = lmp;
     }
     // wrap it with a sorting mp
-    return new SortingMergePolicy(mp, sorter);
+    return new SortingMergePolicy(mp, sort);
   }
 
   private void createRandomIndexes() throws IOException {
@@ -101,7 +103,7 @@ public class TestSortingMergePolicy exte
     final long seed = random().nextLong();
     final IndexWriterConfig iwc1 = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(new Random(seed)));
     final IndexWriterConfig iwc2 = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(new Random(seed)));
-    iwc2.setMergePolicy(newSortingMergePolicy(sorter));
+    iwc2.setMergePolicy(newSortingMergePolicy(sort));
     final RandomIndexWriter iw1 = new RandomIndexWriter(new Random(seed), dir1, iwc1);
     final RandomIndexWriter iw2 = new RandomIndexWriter(new Random(seed), dir2, iwc2);
     for (int i = 0; i < numDocs; ++i) {
@@ -164,7 +166,7 @@ public class TestSortingMergePolicy exte
   }
 
   public void testSortingMP() throws IOException {
-    final AtomicReader sortedReader1 = SortingAtomicReader.wrap(SlowCompositeReaderWrapper.wrap(reader), sorter);
+    final AtomicReader sortedReader1 = SortingAtomicReader.wrap(SlowCompositeReaderWrapper.wrap(reader), sort);
     final AtomicReader sortedReader2 = SlowCompositeReaderWrapper.wrap(sortedReader);
 
     assertSorted(sortedReader1);

Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java?rev=1575253&r1=1575252&r2=1575253&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java Fri Mar  7 13:07:45 2014
@@ -46,17 +46,12 @@ import org.apache.lucene.index.BinaryDoc
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.FilterAtomicReader;
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.MultiDocValues;
-import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SegmentReader;
-import org.apache.lucene.index.SlowCompositeReaderWrapper;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.sorter.EarlyTerminatingSortingCollector;
-import org.apache.lucene.index.sorter.Sorter;
-import org.apache.lucene.index.sorter.SortingAtomicReader;
 import org.apache.lucene.index.sorter.SortingMergePolicy;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
@@ -117,9 +112,8 @@ public class AnalyzingInfixSuggester ext
   /** Analyzer used at index time */
   protected final Analyzer indexAnalyzer;
   final Version matchVersion;
-  private final File indexPath;
+  private final Directory dir;
   final int minPrefixChars;
-  private Directory dir;
 
   /** Used for ongoing NRT additions/updates. */
   private IndexWriter writer;
@@ -131,16 +125,19 @@ public class AnalyzingInfixSuggester ext
    *  PrefixQuery is used (4). */
   public static final int DEFAULT_MIN_PREFIX_CHARS = 4;
 
-  private Sorter sorter;
+  /** How we sort the postings and search results. */
+  private static final Sort SORT = new Sort(new SortField("weight", SortField.Type.LONG, true));
 
   /** Create a new instance, loading from a previously built
-   *  directory, if it exists. */
-  public AnalyzingInfixSuggester(Version matchVersion, File indexPath, Analyzer analyzer) throws IOException {
-    this(matchVersion, indexPath, analyzer, analyzer, DEFAULT_MIN_PREFIX_CHARS);
+   *  directory, if it exists.  Note that {@link #close}
+   *  will also close the provided directory. */
+  public AnalyzingInfixSuggester(Version matchVersion, Directory dir, Analyzer analyzer) throws IOException {
+    this(matchVersion, dir, analyzer, analyzer, DEFAULT_MIN_PREFIX_CHARS);
   }
 
   /** Create a new instance, loading from a previously built
-   *  directory, if it exists.
+   *  directory, if it exists. Note that {@link #close}
+   *  will also close the provided directory.
    *
    *  @param minPrefixChars Minimum number of leading characters
    *     before PrefixQuery is used (default 4).
@@ -148,7 +145,7 @@ public class AnalyzingInfixSuggester ext
    *     ngrams (increasing index size but making lookups
    *     faster).
    */
-  public AnalyzingInfixSuggester(Version matchVersion, File indexPath, Analyzer indexAnalyzer, Analyzer queryAnalyzer, int minPrefixChars) throws IOException {
+  public AnalyzingInfixSuggester(Version matchVersion, Directory dir, Analyzer indexAnalyzer, Analyzer queryAnalyzer, int minPrefixChars) throws IOException {
 
     if (minPrefixChars < 0) {
       throw new IllegalArgumentException("minPrefixChars must be >= 0; got: " + minPrefixChars);
@@ -157,33 +154,29 @@ public class AnalyzingInfixSuggester ext
     this.queryAnalyzer = queryAnalyzer;
     this.indexAnalyzer = indexAnalyzer;
     this.matchVersion = matchVersion;
-    this.indexPath = indexPath;
+    this.dir = dir;
     this.minPrefixChars = minPrefixChars;
-    dir = getDirectory(indexPath);
 
     if (DirectoryReader.indexExists(dir)) {
       // Already built; open it:
-      initSorter();
       writer = new IndexWriter(dir,
-                               getIndexWriterConfig(matchVersion, getGramAnalyzer(), sorter, IndexWriterConfig.OpenMode.APPEND));
+                               getIndexWriterConfig(matchVersion, getGramAnalyzer(), IndexWriterConfig.OpenMode.APPEND));
       searcherMgr = new SearcherManager(writer, true, null);
     }
   }
 
   /** Override this to customize index settings, e.g. which
-   *  codec to use. Sorter is null if this config is for
-   *  the first pass writer. */
-  protected IndexWriterConfig getIndexWriterConfig(Version matchVersion, Analyzer indexAnalyzer, Sorter sorter, IndexWriterConfig.OpenMode openMode) {
+   *  codec to use. */
+  protected IndexWriterConfig getIndexWriterConfig(Version matchVersion, Analyzer indexAnalyzer, IndexWriterConfig.OpenMode openMode) {
     IndexWriterConfig iwc = new IndexWriterConfig(matchVersion, indexAnalyzer);
     iwc.setCodec(new Lucene46Codec());
     iwc.setOpenMode(openMode);
 
-    if (sorter != null) {
-      // This way all merged segments will be sorted at
-      // merge time, allow for per-segment early termination
-      // when those segments are searched:
-      iwc.setMergePolicy(new SortingMergePolicy(iwc.getMergePolicy(), sorter));
-    }
+    // This way all merged segments will be sorted at
+    // merge time, allow for per-segment early termination
+    // when those segments are searched:
+    iwc.setMergePolicy(new SortingMergePolicy(iwc.getMergePolicy(), SORT));
+
     return iwc;
   }
 
@@ -206,16 +199,13 @@ public class AnalyzingInfixSuggester ext
       writer = null;
     }
 
-    Directory dirTmp = getDirectory(new File(indexPath.toString() + ".tmp"));
-
-    IndexWriter w = null;
     AtomicReader r = null;
     boolean success = false;
     try {
       // First pass: build a temporary normal Lucene index,
       // just indexing the suggestions as they iterate:
-      w = new IndexWriter(dirTmp,
-                          getIndexWriterConfig(matchVersion, getGramAnalyzer(), null, IndexWriterConfig.OpenMode.CREATE));
+      writer = new IndexWriter(dir,
+                               getIndexWriterConfig(matchVersion, getGramAnalyzer(), IndexWriterConfig.OpenMode.CREATE));
       BytesRef text;
       Document doc = new Document();
       FieldType ft = getTextFieldType();
@@ -253,37 +243,17 @@ public class AnalyzingInfixSuggester ext
         if (iter.hasPayloads()) {
           payloadField.setBytesValue(iter.payload());
         }
-        w.addDocument(doc);
+        writer.addDocument(doc);
       }
       //System.out.println("initial indexing time: " + ((System.nanoTime()-t0)/1000000) + " msec");
 
-      // Second pass: sort the entire index:
-      r = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(w, false));
-      //long t1 = System.nanoTime();
-
-      // We can rollback the first pass, now that have have
-      // the reader open, because we will discard it anyway
-      // (no sense in fsync'ing it):
-      w.rollback();
-
-      initSorter();
-
-      r = SortingAtomicReader.wrap(r, sorter);
-      
-      writer = new IndexWriter(dir,
-                               getIndexWriterConfig(matchVersion, getGramAnalyzer(), sorter, IndexWriterConfig.OpenMode.CREATE));
-      writer.addIndexes(new IndexReader[] {r});
-      r.close();
-
-      //System.out.println("sort time: " + ((System.nanoTime()-t1)/1000000) + " msec");
-
       searcherMgr = new SearcherManager(writer, true, null);
       success = true;
     } finally {
       if (success) {
-        IOUtils.close(w, r, dirTmp);
+        IOUtils.close(r);
       } else {
-        IOUtils.closeWhileHandlingException(w, writer, r, dirTmp);
+        IOUtils.closeWhileHandlingException(writer, r);
         writer = null;
       }
     }
@@ -359,39 +329,6 @@ public class AnalyzingInfixSuggester ext
     searcherMgr.maybeRefreshBlocking();
   }
 
-  private void initSorter() {
-    sorter = new Sorter() {
-
-        @Override
-        public Sorter.DocMap sort(AtomicReader reader) throws IOException {
-          final NumericDocValues weights = reader.getNumericDocValues("weight");
-          final Sorter.DocComparator comparator = new Sorter.DocComparator() {
-              @Override
-              public int compare(int docID1, int docID2) {
-                final long v1 = weights.get(docID1);
-                final long v2 = weights.get(docID2);
-                // Reverse sort (highest weight first);
-                // java7 only:
-                //return Long.compare(v2, v1);
-                if (v1 > v2) {
-                  return -1;
-                } else if (v1 < v2) {
-                  return 1;
-                } else {
-                  return 0;
-                }
-              }
-            };
-          return Sorter.sort(reader.maxDoc(), comparator);
-        }
-
-        @Override
-        public String getID() {
-          return "BySuggestWeight";
-        }
-      };
-  }
-
   /**
    * Subclass can override this method to change the field type of the text field
    * e.g. to change the index options
@@ -502,12 +439,11 @@ public class AnalyzingInfixSuggester ext
     //System.out.println("finalQuery=" + query);
 
     // Sort by weight, descending:
-    TopFieldCollector c = TopFieldCollector.create(new Sort(new SortField("weight", SortField.Type.LONG, true)),
-                                                   num, true, false, false, false);
+    TopFieldCollector c = TopFieldCollector.create(SORT, num, true, false, false, false);
 
     // We sorted postings by weight during indexing, so we
     // only retrieve the first num hits now:
-    Collector c2 = new EarlyTerminatingSortingCollector(c, sorter, num);
+    Collector c2 = new EarlyTerminatingSortingCollector(c, SORT, num);
     IndexSearcher searcher = searcherMgr.acquire();
     List<LookupResult> results = null;
     try {
@@ -517,7 +453,7 @@ public class AnalyzingInfixSuggester ext
       TopFieldDocs hits = (TopFieldDocs) c.topDocs();
 
       // Slower way if postings are not pre-sorted by weight:
-      // hits = searcher.search(query, null, num, new Sort(new SortField("weight", SortField.Type.LONG, true)));
+      // hits = searcher.search(query, null, num, SORT);
       results = createResults(searcher, hits, num, key, doHighlight, matchedTokens, prefixToken);
     } finally {
       searcherMgr.release(searcher);
@@ -684,11 +620,8 @@ public class AnalyzingInfixSuggester ext
     }
     if (writer != null) {
       writer.close();
-      writer = null;
-    }
-    if (dir != null) {
       dir.close();
-      dir = null;
+      writer = null;
     }
   }
 

Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggester.java?rev=1575253&r1=1575252&r2=1575253&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggester.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggester.java Fri Mar  7 13:07:45 2014
@@ -17,7 +17,6 @@ package org.apache.lucene.search.suggest
  * limitations under the License.
  */
 
-import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Comparator;
@@ -38,6 +37,7 @@ import org.apache.lucene.search.FieldDoc
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.TopFieldDocs;
 import org.apache.lucene.search.suggest.Lookup;
+import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.Version;
 
@@ -92,8 +92,8 @@ public class BlendedInfixSuggester exten
    * Create a new instance, loading from a previously built
    * directory, if it exists.
    */
-  public BlendedInfixSuggester(Version matchVersion, File indexPath, Analyzer analyzer) throws IOException {
-    super(matchVersion, indexPath, analyzer);
+  public BlendedInfixSuggester(Version matchVersion, Directory dir, Analyzer analyzer) throws IOException {
+    super(matchVersion, dir, analyzer);
     this.blenderType = BlenderType.POSITION_LINEAR;
     this.numFactor = DEFAULT_NUM_FACTOR;
   }
@@ -106,9 +106,9 @@ public class BlendedInfixSuggester exten
    * @param numFactor   Factor to multiply the number of searched elements before ponderate
    * @throws IOException If there are problems opening the underlying Lucene index.
    */
-  public BlendedInfixSuggester(Version matchVersion, File indexPath, Analyzer indexAnalyzer, Analyzer queryAnalyzer,
+  public BlendedInfixSuggester(Version matchVersion, Directory dir, Analyzer indexAnalyzer, Analyzer queryAnalyzer,
                                int minPrefixChars, BlenderType blenderType, int numFactor) throws IOException {
-    super(matchVersion, indexPath, indexAnalyzer, queryAnalyzer, minPrefixChars);
+    super(matchVersion, dir, indexAnalyzer, queryAnalyzer, minPrefixChars);
     this.blenderType = blenderType;
     this.numFactor = numFactor;
   }

Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/test/org/apache/lucene/search/suggest/LookupBenchmarkTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/test/org/apache/lucene/search/suggest/LookupBenchmarkTest.java?rev=1575253&r1=1575252&r2=1575253&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/test/org/apache/lucene/search/suggest/LookupBenchmarkTest.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/test/org/apache/lucene/search/suggest/LookupBenchmarkTest.java Fri Mar  7 13:07:45 2014
@@ -40,6 +40,7 @@ import org.apache.lucene.search.suggest.
 import org.apache.lucene.search.suggest.fst.WFSTCompletionLookup;
 import org.apache.lucene.search.suggest.jaspell.JaspellLookup;
 import org.apache.lucene.search.suggest.tst.TSTLookup;
+import org.apache.lucene.store.FSDirectory;
 import org.apache.lucene.util.*;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
@@ -161,7 +162,7 @@ public class LookupBenchmarkTest extends
     } catch (InstantiationException e) {
       Analyzer a = new MockAnalyzer(random, MockTokenizer.KEYWORD, false);
       if (cls == AnalyzingInfixSuggester.class) {
-        lookup = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, TestUtil.getTempDir("LookupBenchmarkTest"), a);
+        lookup = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, FSDirectory.open(TestUtil.getTempDir("LookupBenchmarkTest")), a);
       } else {
         Constructor<? extends Lookup> ctor = cls.getConstructor(Analyzer.class);
         lookup = ctor.newInstance(a);

Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java?rev=1575253&r1=1575252&r2=1575253&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java Fri Mar  7 13:07:45 2014
@@ -22,7 +22,6 @@ import java.io.IOException;
 import java.io.Reader;
 import java.io.StringReader;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashSet;
@@ -40,7 +39,6 @@ import org.apache.lucene.analysis.util.C
 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.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
@@ -57,15 +55,8 @@ public class AnalyzingInfixSuggesterTest
       new Input("a penny saved is a penny earned", 10, new BytesRef("foobaz")),
     };
 
-    File tempDir = TestUtil.getTempDir("AnalyzingInfixSuggesterTest");
-
     Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
-    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, tempDir, a, a, 3) {
-        @Override
-        protected Directory getDirectory(File path) {
-          return newDirectory();
-        }
-      };
+    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), a, a, 3);
     suggester.build(new InputArrayIterator(keys));
 
     List<LookupResult> results = suggester.lookup(TestUtil.stringToCharSequence("ear", random()), 10, true, true);
@@ -108,22 +99,12 @@ public class AnalyzingInfixSuggesterTest
     File tempDir = TestUtil.getTempDir("AnalyzingInfixSuggesterTest");
 
     Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
-    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, tempDir, a, a, 3) {
-        @Override
-        protected Directory getDirectory(File path) {
-          return newFSDirectory(path);
-        }
-      };
+    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a, 3);
     suggester.build(new InputArrayIterator(keys));
     assertEquals(2, suggester.getCount());
     suggester.close();
 
-    suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, tempDir, a, a, 3) {
-        @Override
-        protected Directory getDirectory(File path) {
-          return newFSDirectory(path);
-        }
-      };
+    suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a, 3);
     List<LookupResult> results = suggester.lookup(TestUtil.stringToCharSequence("ear", random()), 10, true, true);
     assertEquals(2, results.size());
     assertEquals("a penny saved is a penny <b>ear</b>ned", results.get(0).key);
@@ -161,15 +142,8 @@ public class AnalyzingInfixSuggesterTest
       new Input("a penny saved is a penny earned", 10, new BytesRef("foobaz")),
     };
 
-    File tempDir = TestUtil.getTempDir("AnalyzingInfixSuggesterTest");
-
     Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
-    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, tempDir, a, a, 3) {
-        @Override
-        protected Directory getDirectory(File path) {
-          return newDirectory();
-        }
-
+    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), a, a, 3) {
         @Override
         protected Object highlight(String text, Set<String> matchedTokens, String prefixToken) throws IOException {
           TokenStream ts = queryAnalyzer.tokenStream("text", new StringReader(text));
@@ -244,17 +218,11 @@ public class AnalyzingInfixSuggesterTest
       new Input("lend me your ear", 8, new BytesRef("foobar")),
       new Input("a penny saved is a penny earned", 10, new BytesRef("foobaz")),
     };
-
     File tempDir = TestUtil.getTempDir("AnalyzingInfixSuggesterTest");
 
     Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
     int minPrefixLength = random().nextInt(10);
-    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, tempDir, a, a, minPrefixLength) {
-        @Override
-        protected Directory getDirectory(File path) {
-          return newFSDirectory(path);
-        }
-      };
+    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a, minPrefixLength);
     suggester.build(new InputArrayIterator(keys));
 
     for(int i=0;i<2;i++) {
@@ -311,12 +279,7 @@ public class AnalyzingInfixSuggesterTest
 
       // Make sure things still work after close and reopen:
       suggester.close();
-      suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, tempDir, a, a, minPrefixLength) {
-          @Override
-          protected Directory getDirectory(File path) {
-            return newFSDirectory(path);
-          }
-        };
+      suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a, minPrefixLength);
     }
     suggester.close();
   }
@@ -326,15 +289,8 @@ public class AnalyzingInfixSuggesterTest
       new Input("a penny saved is a penny earned", 10, new BytesRef("foobaz")),
     };
 
-    File tempDir = TestUtil.getTempDir("AnalyzingInfixSuggesterTest");
-
     Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
-    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, tempDir, a, a, 3) {
-        @Override
-        protected Directory getDirectory(File path) {
-          return newDirectory();
-        }
-      };
+    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), a, a, 3);
     suggester.build(new InputArrayIterator(keys));
     List<LookupResult> results = suggester.lookup(TestUtil.stringToCharSequence("penn", random()), 10, true, true);
     assertEquals(1, results.size());
@@ -347,15 +303,8 @@ public class AnalyzingInfixSuggesterTest
       new Input("a Penny saved is a penny earned", 10, new BytesRef("foobaz")),
     };
 
-    File tempDir = TestUtil.getTempDir("AnalyzingInfixSuggesterTest");
-
     Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, true);
-    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, tempDir, a, a, 3) {
-        @Override
-        protected Directory getDirectory(File path) {
-          return newDirectory();
-        }
-      };
+    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), a, a, 3);
     suggester.build(new InputArrayIterator(keys));
     List<LookupResult> results = suggester.lookup(TestUtil.stringToCharSequence("penn", random()), 10, true, true);
     assertEquals(1, results.size());
@@ -364,18 +313,13 @@ public class AnalyzingInfixSuggesterTest
 
     // Try again, but overriding addPrefixMatch to highlight
     // the entire hit:
-    suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, tempDir, a, a, 3) {
+    suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), a, a, 3) {
         @Override
         protected void addPrefixMatch(StringBuilder sb, String surface, String analyzed, String prefixToken) {
           sb.append("<b>");
           sb.append(surface);
           sb.append("</b>");
         }
-
-        @Override
-        protected Directory getDirectory(File path) {
-          return newDirectory();
-        }
       };
     suggester.build(new InputArrayIterator(keys));
     results = suggester.lookup(TestUtil.stringToCharSequence("penn", random()), 10, true, true);
@@ -389,15 +333,8 @@ public class AnalyzingInfixSuggesterTest
       new Input("a penny saved is a penny earned", 10, new BytesRef("foobaz")),
     };
 
-    File tempDir = TestUtil.getTempDir("AnalyzingInfixSuggesterTest");
-
     Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
-    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, tempDir, a, a, 3) {
-        @Override
-        protected Directory getDirectory(File path) {
-          return newDirectory();
-        }
-      };
+    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), a, a, 3);
     suggester.build(new InputArrayIterator(keys));
     suggester.close();
     suggester.close();
@@ -423,14 +360,7 @@ public class AnalyzingInfixSuggesterTest
         }
       };
 
-    File tempDir = TestUtil.getTempDir("AnalyzingInfixSuggesterTest");
-
-    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, tempDir, indexAnalyzer, queryAnalyzer, 3) {
-        @Override
-        protected Directory getDirectory(File path) {
-          return newDirectory();
-        }
-      };
+    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), indexAnalyzer, queryAnalyzer, 3);
 
     Input keys[] = new Input[] {
       new Input("a bob for apples", 10, new BytesRef("foobaz")),
@@ -444,14 +374,8 @@ public class AnalyzingInfixSuggesterTest
   }
 
   public void testEmptyAtStart() throws Exception {
-    File tempDir = TestUtil.getTempDir("AnalyzingInfixSuggesterTest");
     Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
-    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, tempDir, a, a, 3) {
-        @Override
-        protected Directory getDirectory(File path) {
-          return newDirectory();
-        }
-      };
+    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), a, a, 3);
     suggester.build(new InputArrayIterator(new Input[0]));
     suggester.add(new BytesRef("a penny saved is a penny earned"), 10, new BytesRef("foobaz"));
     suggester.add(new BytesRef("lend me your ear"), 8, new BytesRef("foobar"));
@@ -488,14 +412,8 @@ public class AnalyzingInfixSuggesterTest
   }
 
   public void testBothExactAndPrefix() throws Exception {
-    File tempDir = TestUtil.getTempDir("AnalyzingInfixSuggesterTest");
     Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
-    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, tempDir, a, a, 3) {
-        @Override
-        protected Directory getDirectory(File path) {
-          return newDirectory();
-        }
-      };
+    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), a, a, 3);
     suggester.build(new InputArrayIterator(new Input[0]));
     suggester.add(new BytesRef("the pen is pretty"), 10, new BytesRef("foobaz"));
     suggester.refresh();
@@ -568,12 +486,7 @@ public class AnalyzingInfixSuggesterTest
       System.out.println("  minPrefixChars=" + minPrefixChars);
     }
 
-    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, tempDir, a, a, minPrefixChars) {
-        @Override
-        protected Directory getDirectory(File path) {
-          return newFSDirectory(path);
-        }
-      };
+    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a, minPrefixChars);
 
     // Initial suggester built with nothing:
     suggester.build(new InputArrayIterator(new Input[0]));
@@ -653,12 +566,7 @@ public class AnalyzingInfixSuggesterTest
         }
         lookupThread.finish();
         suggester.close();
-        suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, tempDir, a, a, minPrefixChars) {
-            @Override
-            protected Directory getDirectory(File path) {
-              return newFSDirectory(path);
-            }
-          };
+        suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a, minPrefixChars);
         lookupThread = new LookupThread(suggester);
         lookupThread.start();
 
@@ -829,15 +737,8 @@ public class AnalyzingInfixSuggesterTest
       new Input("lend me your ear", 8, new BytesRef("foobar")),
     };
 
-    File tempDir = TestUtil.getTempDir("AnalyzingInfixSuggesterTest");
-
     Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
-    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, tempDir, a, a, 3) {
-        @Override
-        protected Directory getDirectory(File path) {
-          return newDirectory();
-        }
-      };
+    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), a, a, 3);
     suggester.build(new InputArrayIterator(keys));
 
     List<LookupResult> results = suggester.lookup(TestUtil.stringToCharSequence("ear", random()), 10, true, true);

Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggesterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggesterTest.java?rev=1575253&r1=1575252&r2=1575253&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggesterTest.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggesterTest.java Fri Mar  7 13:07:45 2014
@@ -23,7 +23,6 @@ import org.apache.lucene.analysis.util.C
 import org.apache.lucene.search.suggest.Input;
 import org.apache.lucene.search.suggest.InputArrayIterator;
 import org.apache.lucene.search.suggest.Lookup;
-import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
@@ -49,15 +48,10 @@ public class BlendedInfixSuggesterTest e
     File tempDir = TestUtil.getTempDir("BlendedInfixSuggesterTest");
 
     Analyzer a = new StandardAnalyzer(TEST_VERSION_CURRENT, CharArraySet.EMPTY_SET);
-    BlendedInfixSuggester suggester = new BlendedInfixSuggester(TEST_VERSION_CURRENT, tempDir, a, a,
-        AnalyzingInfixSuggester.DEFAULT_MIN_PREFIX_CHARS,
-        BlendedInfixSuggester.BlenderType.POSITION_LINEAR,
-        BlendedInfixSuggester.DEFAULT_NUM_FACTOR) {
-      @Override
-      protected Directory getDirectory(File path) {
-        return newFSDirectory(path);
-      }
-    };
+    BlendedInfixSuggester suggester = new BlendedInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a,
+                                                                AnalyzingInfixSuggester.DEFAULT_MIN_PREFIX_CHARS,
+                                                                BlendedInfixSuggester.BlenderType.POSITION_LINEAR,
+                                                                BlendedInfixSuggester.DEFAULT_NUM_FACTOR);
     suggester.build(new InputArrayIterator(keys));
 
     // we query for star wars and check that the weight
@@ -94,12 +88,7 @@ public class BlendedInfixSuggesterTest e
     Analyzer a = new StandardAnalyzer(TEST_VERSION_CURRENT, CharArraySet.EMPTY_SET);
 
     // BlenderType.LINEAR is used by default (remove position*10%)
-    BlendedInfixSuggester suggester = new BlendedInfixSuggester(TEST_VERSION_CURRENT, tempDir, a) {
-      @Override
-      protected Directory getDirectory(File path) {
-        return newFSDirectory(path);
-      }
-    };
+    BlendedInfixSuggester suggester = new BlendedInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a);
     suggester.build(new InputArrayIterator(keys));
 
     assertEquals(w, getInResults(suggester, "top", pl, 1));
@@ -109,13 +98,8 @@ public class BlendedInfixSuggesterTest e
     suggester.close();
 
     // BlenderType.RECIPROCAL is using 1/(1+p) * w where w is weight and p the position of the word
-    suggester = new BlendedInfixSuggester(TEST_VERSION_CURRENT, tempDir, a, a,
-        AnalyzingInfixSuggester.DEFAULT_MIN_PREFIX_CHARS, BlendedInfixSuggester.BlenderType.POSITION_RECIPROCAL, 1) {
-      @Override
-      protected Directory getDirectory(File path) {
-        return newFSDirectory(path);
-      }
-    };
+    suggester = new BlendedInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a,
+                                          AnalyzingInfixSuggester.DEFAULT_MIN_PREFIX_CHARS, BlendedInfixSuggester.BlenderType.POSITION_RECIPROCAL, 1);
     suggester.build(new InputArrayIterator(keys));
 
     assertEquals(w, getInResults(suggester, "top", pl, 1));
@@ -145,13 +129,8 @@ public class BlendedInfixSuggesterTest e
     Analyzer a = new StandardAnalyzer(TEST_VERSION_CURRENT, CharArraySet.EMPTY_SET);
 
     // if factor is small, we don't get the expected element
-    BlendedInfixSuggester suggester = new BlendedInfixSuggester(TEST_VERSION_CURRENT, tempDir, a, a,
-        AnalyzingInfixSuggester.DEFAULT_MIN_PREFIX_CHARS, BlendedInfixSuggester.BlenderType.POSITION_RECIPROCAL, 1) {
-      @Override
-      protected Directory getDirectory(File path) {
-        return newFSDirectory(path);
-      }
-    };
+    BlendedInfixSuggester suggester = new BlendedInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a,
+                                                                AnalyzingInfixSuggester.DEFAULT_MIN_PREFIX_CHARS, BlendedInfixSuggester.BlenderType.POSITION_RECIPROCAL, 1);
 
     suggester.build(new InputArrayIterator(keys));
 
@@ -169,13 +148,8 @@ public class BlendedInfixSuggesterTest e
     suggester.close();
 
     // if we increase the factor we have it
-    suggester = new BlendedInfixSuggester(TEST_VERSION_CURRENT, tempDir, a, a,
-        AnalyzingInfixSuggester.DEFAULT_MIN_PREFIX_CHARS, BlendedInfixSuggester.BlenderType.POSITION_RECIPROCAL, 2) {
-      @Override
-      protected Directory getDirectory(File path) {
-        return newFSDirectory(path);
-      }
-    };
+    suggester = new BlendedInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a,
+                                          AnalyzingInfixSuggester.DEFAULT_MIN_PREFIX_CHARS, BlendedInfixSuggester.BlenderType.POSITION_RECIPROCAL, 2);
     suggester.build(new InputArrayIterator(keys));
 
     // we have it
@@ -205,14 +179,9 @@ public class BlendedInfixSuggesterTest e
     Analyzer a = new StandardAnalyzer(TEST_VERSION_CURRENT, CharArraySet.EMPTY_SET);
 
     // if factor is small, we don't get the expected element
-    BlendedInfixSuggester suggester = new BlendedInfixSuggester(TEST_VERSION_CURRENT, tempDir, a, a,
-        AnalyzingInfixSuggester.DEFAULT_MIN_PREFIX_CHARS, BlendedInfixSuggester.BlenderType.POSITION_RECIPROCAL,
-        BlendedInfixSuggester.DEFAULT_NUM_FACTOR) {
-      @Override
-      protected Directory getDirectory(File path) {
-        return newFSDirectory(path);
-      }
-    };
+    BlendedInfixSuggester suggester = new BlendedInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a,
+                                                                AnalyzingInfixSuggester.DEFAULT_MIN_PREFIX_CHARS, BlendedInfixSuggester.BlenderType.POSITION_RECIPROCAL,
+                                                                BlendedInfixSuggester.DEFAULT_NUM_FACTOR);
     suggester.build(new InputArrayIterator(keys));
 
 

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/spelling/suggest/fst/AnalyzingInfixLookupFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/spelling/suggest/fst/AnalyzingInfixLookupFactory.java?rev=1575253&r1=1575252&r2=1575253&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/spelling/suggest/fst/AnalyzingInfixLookupFactory.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/spelling/suggest/fst/AnalyzingInfixLookupFactory.java Fri Mar  7 13:07:45 2014
@@ -24,6 +24,7 @@ import org.apache.lucene.analysis.Analyz
 import org.apache.lucene.search.suggest.Lookup;
 import org.apache.lucene.search.suggest.analyzing.AnalyzingInfixSuggester;
 import org.apache.lucene.search.suggest.analyzing.AnalyzingSuggester;
+import org.apache.lucene.store.FSDirectory;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.schema.FieldType;
@@ -90,7 +91,8 @@ public class AnalyzingInfixLookupFactory
 
     try {
       return new AnalyzingInfixSuggester(core.getSolrConfig().luceneMatchVersion, 
-          new File(indexPath), indexAnalyzer, queryAnalyzer, minPrefixChars);
+                                         FSDirectory.open(new File(indexPath)), indexAnalyzer,
+                                         queryAnalyzer, minPrefixChars);
     } catch (IOException e) {
       throw new RuntimeException();
     }

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/spelling/suggest/fst/BlendedInfixLookupFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/spelling/suggest/fst/BlendedInfixLookupFactory.java?rev=1575253&r1=1575252&r2=1575253&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/spelling/suggest/fst/BlendedInfixLookupFactory.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/spelling/suggest/fst/BlendedInfixLookupFactory.java Fri Mar  7 13:07:45 2014
@@ -23,8 +23,9 @@ import java.io.IOException;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.search.suggest.Lookup;
 import org.apache.lucene.search.suggest.analyzing.AnalyzingInfixSuggester;
-import org.apache.lucene.search.suggest.analyzing.BlendedInfixSuggester;
 import org.apache.lucene.search.suggest.analyzing.BlendedInfixSuggester.BlenderType;
+import org.apache.lucene.search.suggest.analyzing.BlendedInfixSuggester;
+import org.apache.lucene.store.FSDirectory;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.schema.FieldType;
@@ -94,7 +95,9 @@ public class BlendedInfixLookupFactory e
     
     try {
       return new BlendedInfixSuggester(core.getSolrConfig().luceneMatchVersion, 
-          new File(indexPath), indexAnalyzer, queryAnalyzer, minPrefixChars, blenderType, numFactor);
+                                       FSDirectory.open(new File(indexPath)),
+                                       indexAnalyzer, queryAnalyzer, minPrefixChars,
+                                       blenderType, numFactor);
     } catch (IOException e) {
       throw new RuntimeException();
     }