You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2015/01/06 16:15:43 UTC

svn commit: r1649824 [1/4] - in /lucene/dev/branches/branch_5x: ./ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/search/ lucene/core/src/test/org/apache/lucene/search/ lucene/expressions/ lucene/expressions/src/java/org/apache/lucene/expr...

Author: jpountz
Date: Tue Jan  6 15:15:42 2015
New Revision: 1649824

URL: http://svn.apache.org/r1649824
Log:
LUCENE-5702: Move comparators to a per-leaf API.

Added:
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/LeafFieldComparator.java
      - copied unchanged from r1649818, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/LeafFieldComparator.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/SimpleFieldComparator.java
      - copied unchanged from r1649818, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/SimpleFieldComparator.java
Modified:
    lucene/dev/branches/branch_5x/   (props changed)
    lucene/dev/branches/branch_5x/lucene/   (props changed)
    lucene/dev/branches/branch_5x/lucene/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_5x/lucene/MIGRATE.txt
    lucene/dev/branches/branch_5x/lucene/core/   (props changed)
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FieldValueHitQueue.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/SortRescorer.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/TopDocsCollector.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java
    lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/JustCompileSearch.java
    lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestElevationComparator.java
    lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestSort.java   (contents, props changed)
    lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java
    lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollector.java   (contents, props changed)
    lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestTopScoreDocCollector.java
    lucene/dev/branches/branch_5x/lucene/expressions/   (props changed)
    lucene/dev/branches/branch_5x/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionComparator.java
    lucene/dev/branches/branch_5x/lucene/grouping/   (props changed)
    lucene/dev/branches/branch_5x/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractFirstPassGroupingCollector.java
    lucene/dev/branches/branch_5x/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractSecondPassGroupingCollector.java
    lucene/dev/branches/branch_5x/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java
    lucene/dev/branches/branch_5x/lucene/grouping/src/java/org/apache/lucene/search/grouping/SearchGroup.java
    lucene/dev/branches/branch_5x/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupHeadsCollector.java
    lucene/dev/branches/branch_5x/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java
    lucene/dev/branches/branch_5x/lucene/join/   (props changed)
    lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinCollector.java
    lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinFieldComparator.java
    lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinSortField.java
    lucene/dev/branches/branch_5x/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java
    lucene/dev/branches/branch_5x/lucene/misc/   (props changed)
    lucene/dev/branches/branch_5x/lucene/misc/src/java/org/apache/lucene/index/Sorter.java
    lucene/dev/branches/branch_5x/lucene/misc/src/java/org/apache/lucene/search/BlockJoinComparatorSource.java
    lucene/dev/branches/branch_5x/lucene/queries/   (props changed)
    lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSource.java
    lucene/dev/branches/branch_5x/lucene/sandbox/   (props changed)
    lucene/dev/branches/branch_5x/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/SlowCollatedStringComparator.java
    lucene/dev/branches/branch_5x/solr/   (props changed)
    lucene/dev/branches/branch_5x/solr/core/   (props changed)
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/RandomSortField.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/ExportQParserPlugin.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/ReRankQParserPlugin.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/search/TestRankQueryPlugin.java

Modified: lucene/dev/branches/branch_5x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/CHANGES.txt?rev=1649824&r1=1649823&r2=1649824&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_5x/lucene/CHANGES.txt Tue Jan  6 15:15:42 2015
@@ -199,6 +199,9 @@ API Changes
 * LUCENE-5527: The Collector API has been refactored to use a dedicated Collector
   per leaf. (Shikhar Bhushan, Adrien Grand)
 
+* LUCENE-5702: The FieldComparator API has been refactor to a per-leaf API, just
+  like Collectors. (Adrien Grand)
+
 * LUCENE-4246: IndexWriter.close now always closes, even if it throws
   an exception.  The new IndexWriterConfig.setCommitOnClose (default
   true) determines whether close() should commit before closing.

Modified: lucene/dev/branches/branch_5x/lucene/MIGRATE.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/MIGRATE.txt?rev=1649824&r1=1649823&r2=1649824&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/MIGRATE.txt (original)
+++ lucene/dev/branches/branch_5x/lucene/MIGRATE.txt Tue Jan  6 15:15:42 2015
@@ -58,6 +58,14 @@ per segment. It is possible to migrate e
 extending SimpleCollector instead of Collector: SimpleCollector is a
 specialization of Collector that returns itself as a per-segment Collector.
 
+## Refactored FieldComparator API (LUCENE-5702)
+
+Like collectors (see above), field comparators have been refactored to
+produce a new comparator (called LeafFieldComparator) per segment. It is
+possible to migrate existing comparators painlessly by extending
+SimpleFieldComparator, which will implements both FieldComparator and
+LeafFieldComparator and return itself as a per-segment comparator.
+
 ## Removed ChainedFilter (LUCENE-5984)
 
 Users are advised to switch to BooleanFilter instead.

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java?rev=1649824&r1=1649823&r2=1649824&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java Tue Jan  6 15:15:42 2015
@@ -17,11 +17,12 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
+
 import java.io.IOException;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.util.Bits;
@@ -38,46 +39,24 @@ import org.apache.lucene.util.BytesRefBu
  * sorting, by exposing a tight interaction with {@link
  * FieldValueHitQueue} as it visits hits.  Whenever a hit is
  * competitive, it's enrolled into a virtual slot, which is
- * an int ranging from 0 to numHits-1.  The {@link
- * FieldComparator} is made aware of segment transitions
- * during searching in case any internal state it's tracking
- * needs to be recomputed during these transitions.</p>
- *
- * <p>A comparator must define these functions:</p>
- *
+ * an int ranging from 0 to numHits-1. Segment transitions are
+ * handled by creating a dedicated per-segment
+ * {@link LeafFieldComparator} which also needs to interact
+ * with the {@link FieldValueHitQueue} but can optimize based
+ * on the segment to collect.</p>
+ * 
+ * <p>The following functions need to be implemented</p>
  * <ul>
- *
  *  <li> {@link #compare} Compare a hit at 'slot a'
  *       with hit 'slot b'.
- *
- *  <li> {@link #setBottom} This method is called by
- *       {@link FieldValueHitQueue} to notify the
- *       FieldComparator of the current weakest ("bottom")
- *       slot.  Note that this slot may not hold the weakest
- *       value according to your comparator, in cases where
- *       your comparator is not the primary one (ie, is only
- *       used to break ties from the comparators before it).
- *
- *  <li> {@link #compareBottom} Compare a new hit (docID)
- *       against the "weakest" (bottom) entry in the queue.
- *
+ * 
  *  <li> {@link #setTopValue} This method is called by
  *       {@link TopFieldCollector} to notify the
  *       FieldComparator of the top most value, which is
- *       used by future calls to {@link #compareTop}.
- *
- *  <li> {@link #compareBottom} Compare a new hit (docID)
- *       against the "weakest" (bottom) entry in the queue.
- *
- *  <li> {@link #compareTop} Compare a new hit (docID)
- *       against the top value previously set by a call to
- *       {@link #setTopValue}.
- *
- *  <li> {@link #copy} Installs a new hit into the
- *       priority queue.  The {@link FieldValueHitQueue}
- *       calls this method when a new hit is competitive.
- *
- *  <li> {@link #setNextReader(org.apache.lucene.index.LeafReaderContext)} Invoked
+ *       used by future calls to
+ *       {@link LeafFieldComparator#compareTop}.
+ * 
+ *  <li> {@link #getLeafComparator(org.apache.lucene.index.LeafReaderContext)} Invoked
  *       when the search is switching to the next segment.
  *       You may need to update internal state of the
  *       comparator, for example retrieving new values from
@@ -89,6 +68,7 @@ import org.apache.lucene.util.BytesRefBu
  *       FieldDoc#fields} when returning the top results.
  * </ul>
  *
+ * @see LeafFieldComparator
  * @lucene.experimental
  */
 public abstract class FieldComparator<T> {
@@ -105,101 +85,35 @@ public abstract class FieldComparator<T>
   public abstract int compare(int slot1, int slot2);
 
   /**
-   * Set the bottom slot, ie the "weakest" (sorted last)
-   * entry in the queue.  When {@link #compareBottom} is
-   * called, you should compare against this slot.  This
-   * will always be called before {@link #compareBottom}.
-   * 
-   * @param slot the currently weakest (sorted last) slot in the queue
-   */
-  public abstract void setBottom(final int slot);
-
-  /**
    * Record the top value, for future calls to {@link
-   * #compareTop}.  This is only called for searches that
+   * LeafFieldComparator#compareTop}.  This is only called for searches that
    * use searchAfter (deep paging), and is called before any
-   * calls to {@link #setNextReader}.
+   * calls to {@link #getLeafComparator(LeafReaderContext)}.
    */
   public abstract void setTopValue(T value);
 
   /**
-   * Compare the bottom of the queue with this doc.  This will
-   * only invoked after setBottom has been called.  This
-   * should return the same result as {@link
-   * #compare(int,int)}} as if bottom were slot1 and the new
-   * document were slot 2.
-   *    
-   * <p>For a search that hits many results, this method
-   * will be the hotspot (invoked by far the most
-   * frequently).</p>
-   * 
-   * @param doc that was hit
-   * @return any {@code N < 0} if the doc's value is sorted after
-   * the bottom entry (not competitive), any {@code N > 0} if the
-   * doc's value is sorted before the bottom entry and {@code 0} if
-   * they are equal.
-   */
-  public abstract int compareBottom(int doc) throws IOException;
-
-  /**
-   * Compare the top value with this doc.  This will
-   * only invoked after setTopValue has been called.  This
-   * should return the same result as {@link
-   * #compare(int,int)}} as if topValue were slot1 and the new
-   * document were slot 2.  This is only called for searches that
-   * use searchAfter (deep paging).
-   *    
-   * @param doc that was hit
-   * @return any {@code N < 0} if the doc's value is sorted after
-   * the bottom entry (not competitive), any {@code N > 0} if the
-   * doc's value is sorted before the bottom entry and {@code 0} if
-   * they are equal.
-   */
-  public abstract int compareTop(int doc) throws IOException;
-
-  /**
-   * This method is called when a new hit is competitive.
-   * You should copy any state associated with this document
-   * that will be required for future comparisons, into the
-   * specified slot.
-   * 
-   * @param slot which slot to copy the hit to
-   * @param doc docID relative to current reader
+   * Return the actual value in the slot.
+   *
+   * @param slot the value
+   * @return value in this slot
    */
-  public abstract void copy(int slot, int doc) throws IOException;
+  public abstract T value(int slot);
 
   /**
-   * Set a new {@link org.apache.lucene.index.LeafReaderContext}. All subsequent docIDs are relative to
-   * the current reader (you must add docBase if you need to
-   * map it to a top-level docID).
+   * Get a per-segment {@link LeafFieldComparator} to collect the given
+   * {@link org.apache.lucene.index.LeafReaderContext}. All docIDs supplied to
+   * this {@link LeafFieldComparator} are relative to the current reader (you
+   * must add docBase if you need to map it to a top-level docID).
    * 
    * @param context current reader context
-   * @return the comparator to use for this segment; most
-   *   comparators can just return "this" to reuse the same
-   *   comparator across segments
+   * @return the comparator to use for this segment
    * @throws IOException if there is a low-level IO error
    */
-  public abstract FieldComparator<T> setNextReader(LeafReaderContext context) throws IOException;
-
-  /** Sets the Scorer to use in case a document's score is
-   *  needed.
-   * 
-   * @param scorer Scorer instance that you should use to
-   * obtain the current hit's score, if necessary. */
-  public void setScorer(Scorer scorer) {
-    // Empty implementation since most comparators don't need the score. This
-    // can be overridden by those that need it.
-  }
-  
-  /**
-   * Return the actual value in the slot.
-   *
-   * @param slot the value
-   * @return value in this slot
-   */
-  public abstract T value(int slot);
+  public abstract LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException;
 
-  /** Returns -1 if first is less than second.  Default
+  /** Returns a negative integer if first is less than second,
+   *  0 if they are equal and a positive integer otherwise. Default
    *  impl to assume the type implements Comparable and
    *  invoke .compareTo; be sure to override this method if
    *  your FieldComparator's type isn't a Comparable or
@@ -219,10 +133,11 @@ public abstract class FieldComparator<T>
     }
   }
 
+
   /**
    * Base FieldComparator class for numeric types
    */
-  public static abstract class NumericComparator<T extends Number> extends FieldComparator<T> {
+  public static abstract class NumericComparator<T extends Number> extends SimpleFieldComparator<T> {
     protected final T missingValue;
     protected final String field;
     protected Bits docsWithField;
@@ -234,7 +149,7 @@ public abstract class FieldComparator<T>
     }
 
     @Override
-    public FieldComparator<T> setNextReader(LeafReaderContext context) throws IOException {
+    protected void doSetNextReader(LeafReaderContext context) throws IOException {
       currentReaderValues = getNumericDocValues(context, field);
       if (missingValue != null) {
         docsWithField = DocValues.getDocsWithField(context.reader(), field);
@@ -245,7 +160,6 @@ public abstract class FieldComparator<T>
       } else {
         docsWithField = null;
       }
-      return this;
     }
     
     /** Retrieves the NumericDocValues for the field in this segment */
@@ -551,7 +465,7 @@ public abstract class FieldComparator<T>
    *  using {@link TopScoreDocCollector} directly (which {@link
    *  IndexSearcher#search} uses when no {@link Sort} is
    *  specified). */
-  public static final class RelevanceComparator extends FieldComparator<Float> {
+  public static final class RelevanceComparator extends FieldComparator<Float> implements LeafFieldComparator {
     private final float[] scores;
     private float bottom;
     private Scorer scorer;
@@ -581,7 +495,7 @@ public abstract class FieldComparator<T>
     }
 
     @Override
-    public FieldComparator<Float> setNextReader(LeafReaderContext context) {
+    public LeafFieldComparator getLeafComparator(LeafReaderContext context) {
       return this;
     }
     
@@ -629,7 +543,7 @@ public abstract class FieldComparator<T>
   }
 
   /** Sorts by ascending docID */
-  public static final class DocComparator extends FieldComparator<Integer> {
+  public static final class DocComparator extends FieldComparator<Integer> implements LeafFieldComparator {
     private final int[] docIDs;
     private int docBase;
     private int bottom;
@@ -658,7 +572,7 @@ public abstract class FieldComparator<T>
     }
 
     @Override
-    public FieldComparator<Integer> setNextReader(LeafReaderContext context) {
+    public LeafFieldComparator getLeafComparator(LeafReaderContext context) {
       // TODO: can we "map" our docIDs to the current
       // reader? saves having to then subtract on every
       // compare call
@@ -686,6 +600,9 @@ public abstract class FieldComparator<T>
       int docValue = docBase + doc;
       return Integer.compare(topValue, docValue);
     }
+
+    @Override
+    public void setScorer(Scorer scorer) {}
   }
   
   /** Sorts by field's natural Term sort order, using
@@ -697,7 +614,7 @@ public abstract class FieldComparator<T>
    *  to large results, this comparator will be much faster
    *  than {@link org.apache.lucene.search.FieldComparator.TermValComparator}.  For very small
    *  result sets it may be slower. */
-  public static class TermOrdValComparator extends FieldComparator<BytesRef> {
+  public static class TermOrdValComparator extends FieldComparator<BytesRef> implements LeafFieldComparator {
     /* Ords for each slot.
        @lucene.internal */
     final int[] ords;
@@ -841,7 +758,7 @@ public abstract class FieldComparator<T>
     }
     
     @Override
-    public FieldComparator<BytesRef> setNextReader(LeafReaderContext context) throws IOException {
+    public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
       termsIndex = getSortedDocValues(context, field);
       currentReaderGen++;
 
@@ -859,7 +776,7 @@ public abstract class FieldComparator<T>
         topOrd = missingOrd;
         topSameReader = true;
       }
-      //System.out.println("  setNextReader topOrd=" + topOrd + " topSameReader=" + topSameReader);
+      //System.out.println("  getLeafComparator topOrd=" + topOrd + " topSameReader=" + topSameReader);
 
       if (bottomSlot != -1) {
         // Recompute bottomOrd/SameReader
@@ -947,13 +864,16 @@ public abstract class FieldComparator<T>
       }
       return val1.compareTo(val2);
     }
+
+    @Override
+    public void setScorer(Scorer scorer) {}
   }
   
   /** Sorts by field's natural Term sort order.  All
    *  comparisons are done using BytesRef.compareTo, which is
    *  slow for medium to large result sets but possibly
    *  very fast for very small results sets. */
-  public static class TermValComparator extends FieldComparator<BytesRef> {
+  public static class TermValComparator extends FieldComparator<BytesRef> implements LeafFieldComparator {
     
     private final BytesRef[] values;
     private final BytesRefBuilder[] tempBRs;
@@ -1019,7 +939,7 @@ public abstract class FieldComparator<T>
     }
 
     @Override
-    public FieldComparator<BytesRef> setNextReader(LeafReaderContext context) throws IOException {
+    public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
       docTerms = getBinaryDocValues(context, field);
       docsWithField = getDocsWithField(context, field);
       if (docsWithField instanceof Bits.MatchAllBits) {
@@ -1075,5 +995,8 @@ public abstract class FieldComparator<T>
       }
       return term;
     }
+
+    @Override
+    public void setScorer(Scorer scorer) {}
   }
 }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FieldValueHitQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FieldValueHitQueue.java?rev=1649824&r1=1649823&r2=1649824&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FieldValueHitQueue.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FieldValueHitQueue.java Tue Jan  6 15:15:42 2015
@@ -19,6 +19,7 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.util.PriorityQueue;
 
 /**
@@ -53,17 +54,17 @@ public abstract class FieldValueHitQueue
    * there is just one comparator.
    */
   private static final class OneComparatorFieldValueHitQueue<T extends FieldValueHitQueue.Entry> extends FieldValueHitQueue<T> {
+    
     private final int oneReverseMul;
+    private final FieldComparator<?> oneComparator;
     
     public OneComparatorFieldValueHitQueue(SortField[] fields, int size)
         throws IOException {
       super(fields, size);
 
-      SortField field = fields[0];
-      setComparator(0,field.getComparator(size, 0));
-      oneReverseMul = field.reverse ? -1 : 1;
-
-      reverseMul[0] = oneReverseMul;
+      assert fields.length == 1;
+      oneComparator = comparators[0];
+      oneReverseMul = reverseMul[0];
     }
 
     /**
@@ -78,7 +79,7 @@ public abstract class FieldValueHitQueue
       assert hitA != hitB;
       assert hitA.slot != hitB.slot;
 
-      final int c = oneReverseMul * firstComparator.compare(hitA.slot, hitB.slot);
+      final int c = oneReverseMul * oneComparator.compare(hitA.slot, hitB.slot);
       if (c != 0) {
         return c > 0;
       }
@@ -98,14 +99,6 @@ public abstract class FieldValueHitQueue
     public MultiComparatorsFieldValueHitQueue(SortField[] fields, int size)
         throws IOException {
       super(fields, size);
-
-      int numComparators = comparators.length;
-      for (int i = 0; i < numComparators; ++i) {
-        SortField field = fields[i];
-
-        reverseMul[i] = field.reverse ? -1 : 1;
-        setComparator(i, field.getComparator(size, i));
-      }
     }
   
     @Override
@@ -130,8 +123,7 @@ public abstract class FieldValueHitQueue
   }
   
   // prevent instantiation and extension.
-  @SuppressWarnings({"rawtypes","unchecked"})
-  private FieldValueHitQueue(SortField[] fields, int size) {
+  private FieldValueHitQueue(SortField[] fields, int size) throws IOException {
     super(size);
     // When we get here, fields.length is guaranteed to be > 0, therefore no
     // need to check it again.
@@ -141,8 +133,14 @@ public abstract class FieldValueHitQueue
     // anyway.
     this.fields = fields;
     int numComparators = fields.length;
-    comparators = new FieldComparator[numComparators];
+    comparators = new FieldComparator<?>[numComparators];
     reverseMul = new int[numComparators];
+    for (int i = 0; i < numComparators; ++i) {
+      SortField field = fields[i];
+
+      reverseMul[i] = field.reverse ? -1 : 1;
+      comparators[i] = field.getComparator(size, i);
+    }
   }
 
   /**
@@ -179,15 +177,17 @@ public abstract class FieldValueHitQueue
     return reverseMul;
   }
 
-  public void setComparator(int pos, FieldComparator<?> comparator) {
-    if (pos==0) firstComparator = comparator;
-    comparators[pos] = comparator;
+  public LeafFieldComparator[] getComparators(LeafReaderContext context) throws IOException {
+    LeafFieldComparator[] comparators = new LeafFieldComparator[this.comparators.length];
+    for (int i = 0; i < comparators.length; ++i) {
+      comparators[i] = this.comparators[i].getLeafComparator(context);
+    }
+    return comparators;
   }
 
   /** Stores the sort criteria being used. */
   protected final SortField[] fields;
-  protected final FieldComparator<?>[] comparators;  // use setComparator to change this array
-  protected FieldComparator<?> firstComparator;      // this must always be equal to comparators[0]
+  protected final FieldComparator<?>[] comparators;
   protected final int[] reverseMul;
 
   @Override

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/SortRescorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/SortRescorer.java?rev=1649824&r1=1649823&r2=1649824&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/SortRescorer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/SortRescorer.java Tue Jan  6 15:15:42 2015
@@ -61,6 +61,7 @@ public class SortRescorer extends Rescor
     int endDoc = 0;
     int docBase = 0;
 
+    LeafCollector leafCollector = null;
     FakeScorer fakeScorer = new FakeScorer();
 
     while (hitUpto < hits.length) {
@@ -75,15 +76,15 @@ public class SortRescorer extends Rescor
 
       if (readerContext != null) {
         // We advanced to another segment:
-        collector.getLeafCollector(readerContext);
-        collector.setScorer(fakeScorer);
+        leafCollector = collector.getLeafCollector(readerContext);
+        leafCollector.setScorer(fakeScorer);
         docBase = readerContext.docBase;
       }
 
       fakeScorer.score = hit.score;
       fakeScorer.doc = docID - docBase;
 
-      collector.collect(fakeScorer.doc);
+      leafCollector.collect(fakeScorer.doc);
 
       hitUpto++;
     }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/TopDocsCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/TopDocsCollector.java?rev=1649824&r1=1649823&r2=1649824&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/TopDocsCollector.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/TopDocsCollector.java Tue Jan  6 15:15:42 2015
@@ -31,7 +31,7 @@ import org.apache.lucene.util.PriorityQu
  * however, you might want to consider overriding all methods, in order to avoid
  * a NullPointerException.
  */
-public abstract class TopDocsCollector<T extends ScoreDoc> extends SimpleCollector {
+public abstract class TopDocsCollector<T extends ScoreDoc> implements Collector {
 
   /** This is used in case topDocs() is called with illegal parameters, or there
    *  simply aren't (enough) results. */