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:03:20 UTC

svn commit: r1649818 [2/4] - in /lucene/dev/trunk: lucene/ lucene/core/src/java/org/apache/lucene/search/ lucene/core/src/test/org/apache/lucene/search/ lucene/expressions/src/java/org/apache/lucene/expressions/ lucene/grouping/src/java/org/apache/luce...

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java?rev=1649818&r1=1649817&r2=1649818&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java Tue Jan  6 15:03:19 2015
@@ -29,814 +29,692 @@ import org.apache.lucene.util.PriorityQu
  * <p/>
  * See the {@link #create(org.apache.lucene.search.Sort, int, boolean, boolean, boolean, boolean)} method
  * for instantiating a TopFieldCollector.
- * 
+ *
  * @lucene.experimental
  */
 public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
-  
+
   // TODO: one optimization we could do is to pre-fill
   // the queue with sentinel value that guaranteed to
   // always compare lower than a real hit; this would
   // save having to check queueFull on each insert
 
-  /*
-   * Implements a TopFieldCollector over one SortField criteria, without
-   * tracking document scores and maxScore.
-   */
-  private static class OneComparatorNonScoringCollector extends 
-      TopFieldCollector {
+  private static abstract class OneComparatorLeafCollector implements LeafCollector {
 
-    FieldComparator<?> comparator;
+    final LeafFieldComparator comparator;
     final int reverseMul;
-    final FieldValueHitQueue<Entry> queue;
-    
-    public OneComparatorNonScoringCollector(FieldValueHitQueue<Entry> queue,
-        int numHits, boolean fillFields) {
-      super(queue, numHits, fillFields);
-      this.queue = queue;
-      comparator = queue.getComparators()[0];
-      reverseMul = queue.getReverseMul()[0];
-    }
-    
-    final void updateBottom(int doc) {
-      // bottom.score is already set to Float.NaN in add().
-      bottom.doc = docBase + doc;
-      bottom = pq.updateTop();
-    }
+    Scorer scorer;
 
-    @Override
-    public void collect(int doc) throws IOException {
-      ++totalHits;
-      if (queueFull) {
-        if ((reverseMul * comparator.compareBottom(doc)) <= 0) {
-          // since docs are visited in doc Id order, if compare is 0, it means
-          // this document is larger than anything else in the queue, and
-          // therefore not competitive.
-          return;
-        }
-        
-        // This hit is competitive - replace bottom element in queue & adjustTop
-        comparator.copy(bottom.slot, doc);
-        updateBottom(doc);
-        comparator.setBottom(bottom.slot);
-      } else {
-        // Startup transient: queue hasn't gathered numHits yet
-        final int slot = totalHits - 1;
-        // Copy hit into queue
-        comparator.copy(slot, doc);
-        add(slot, doc, Float.NaN);
-        if (queueFull) {
-          comparator.setBottom(bottom.slot);
-        }
-      }
+    OneComparatorLeafCollector(LeafFieldComparator comparator, int reverseMul) {
+      this.comparator = comparator;
+      this.reverseMul = reverseMul;
     }
-    
-    @Override
-    protected void doSetNextReader(LeafReaderContext context) throws IOException {
-      this.docBase = context.docBase;
-      queue.setComparator(0, comparator.setNextReader(context));
-      comparator = queue.firstComparator;
-    }
-    
+
     @Override
     public void setScorer(Scorer scorer) throws IOException {
+      this.scorer = scorer;
       comparator.setScorer(scorer);
     }
-    
-  }
 
-  /*
-   * Implements a TopFieldCollector over one SortField criteria, without
-   * tracking document scores and maxScore, and assumes out of orderness in doc
-   * Ids collection.
-   */
-  private static class OutOfOrderOneComparatorNonScoringCollector extends
-      OneComparatorNonScoringCollector {
-
-    public OutOfOrderOneComparatorNonScoringCollector(FieldValueHitQueue<Entry> queue,
-        int numHits, boolean fillFields) {
-      super(queue, numHits, fillFields);
-    }
-    
-    @Override
-    public void collect(int doc) throws IOException {
-      ++totalHits;
-      if (queueFull) {
-        // Fastmatch: return if this hit is not competitive
-        final int cmp = reverseMul * comparator.compareBottom(doc);
-        if (cmp < 0 || (cmp == 0 && doc + docBase > bottom.doc)) {
-          return;
-        }
-        
-        // This hit is competitive - replace bottom element in queue & adjustTop
-        comparator.copy(bottom.slot, doc);
-        updateBottom(doc);
-        comparator.setBottom(bottom.slot);
-      } else {
-        // Startup transient: queue hasn't gathered numHits yet
-        final int slot = totalHits - 1;
-        // Copy hit into queue
-        comparator.copy(slot, doc);
-        add(slot, doc, Float.NaN);
-        if (queueFull) {
-          comparator.setBottom(bottom.slot);
-        }
-      }
-    }
-    
     @Override
     public boolean acceptsDocsOutOfOrder() {
-      return true;
+      return false;
     }
-
   }
 
-  /*
-   * Implements a TopFieldCollector over one SortField criteria, while tracking
-   * document scores but no maxScore.
-   */
-  private static class OneComparatorScoringNoMaxScoreCollector extends
-      OneComparatorNonScoringCollector {
+  private static abstract class MultiComparatorLeafCollector implements LeafCollector {
 
+    final LeafFieldComparator[] comparators;
+    final int[] reverseMul;
+    final LeafFieldComparator firstComparator;
+    final int firstReverseMul;
     Scorer scorer;
 
-    public OneComparatorScoringNoMaxScoreCollector(FieldValueHitQueue<Entry> queue,
-        int numHits, boolean fillFields) {
-      super(queue, numHits, fillFields);
-    }
-    
-    final void updateBottom(int doc, float score) {
-      bottom.doc = docBase + doc;
-      bottom.score = score;
-      bottom = pq.updateTop();
+    MultiComparatorLeafCollector(LeafFieldComparator[] comparators, int[] reverseMul) {
+      this.comparators = comparators;
+      this.reverseMul = reverseMul;
+      firstComparator = comparators[0];
+      firstReverseMul = reverseMul[0];
     }
 
-    @Override
-    public void collect(int doc) throws IOException {
-      ++totalHits;
-      if (queueFull) {
-        if ((reverseMul * comparator.compareBottom(doc)) <= 0) {
-          // since docs are visited in doc Id order, if compare is 0, it means
-          // this document is largest than anything else in the queue, and
-          // therefore not competitive.
-          return;
-        }
-        
-        // Compute the score only if the hit is competitive.
-        final float score = scorer.score();
-
-        // This hit is competitive - replace bottom element in queue & adjustTop
-        comparator.copy(bottom.slot, doc);
-        updateBottom(doc, score);
-        comparator.setBottom(bottom.slot);
-      } else {
-        // Compute the score only if the hit is competitive.
-        final float score = scorer.score();
-
-        // Startup transient: queue hasn't gathered numHits yet
-        final int slot = totalHits - 1;
-        // Copy hit into queue
-        comparator.copy(slot, doc);
-        add(slot, doc, score);
-        if (queueFull) {
-          comparator.setBottom(bottom.slot);
+    protected final int compareBottom(int doc) throws IOException {
+      int cmp = firstReverseMul * firstComparator.compareBottom(doc);
+      if (cmp != 0) {
+        return cmp;
+      }
+      for (int i = 1; i < comparators.length; ++i) {
+        cmp = reverseMul[i] * comparators[i].compareBottom(doc);
+        if (cmp != 0) {
+          return cmp;
         }
       }
+      return 0;
     }
-    
-    @Override
-    public void setScorer(Scorer scorer) throws IOException {
-      this.scorer = scorer;
-      comparator.setScorer(scorer);
-    }
-    
-  }
 
-  /*
-   * Implements a TopFieldCollector over one SortField criteria, while tracking
-   * document scores but no maxScore, and assumes out of orderness in doc Ids
-   * collection.
-   */
-  private static class OutOfOrderOneComparatorScoringNoMaxScoreCollector extends
-      OneComparatorScoringNoMaxScoreCollector {
-
-    public OutOfOrderOneComparatorScoringNoMaxScoreCollector(
-        FieldValueHitQueue<Entry> queue, int numHits, boolean fillFields) {
-      super(queue, numHits, fillFields);
-    }
-    
-    @Override
-    public void collect(int doc) throws IOException {
-      ++totalHits;
-      if (queueFull) {
-        // Fastmatch: return if this hit is not competitive
-        final int cmp = reverseMul * comparator.compareBottom(doc);
-        if (cmp < 0 || (cmp == 0 && doc + docBase > bottom.doc)) {
-          return;
-        }
-        
-        // Compute the score only if the hit is competitive.
-        final float score = scorer.score();
-
-        // This hit is competitive - replace bottom element in queue & adjustTop
-        comparator.copy(bottom.slot, doc);
-        updateBottom(doc, score);
-        comparator.setBottom(bottom.slot);
-      } else {
-        // Compute the score only if the hit is competitive.
-        final float score = scorer.score();
-
-        // Startup transient: queue hasn't gathered numHits yet
-        final int slot = totalHits - 1;
-        // Copy hit into queue
+    protected final void copy(int slot, int doc) throws IOException {
+      for (LeafFieldComparator comparator : comparators) {
         comparator.copy(slot, doc);
-        add(slot, doc, score);
-        if (queueFull) {
-          comparator.setBottom(bottom.slot);
-        }
       }
     }
-    
-    @Override
-    public boolean acceptsDocsOutOfOrder() {
-      return true;
-    }
-
-  }
-
-  /*
-   * Implements a TopFieldCollector over one SortField criteria, with tracking
-   * document scores and maxScore.
-   */
-  private static class OneComparatorScoringMaxScoreCollector extends
-      OneComparatorNonScoringCollector {
 
-    Scorer scorer;
-    
-    public OneComparatorScoringMaxScoreCollector(FieldValueHitQueue<Entry> queue,
-        int numHits, boolean fillFields) {
-      super(queue, numHits, fillFields);
-      // Must set maxScore to NEG_INF, or otherwise Math.max always returns NaN.
-      maxScore = Float.NEGATIVE_INFINITY;
-    }
-    
-    final void updateBottom(int doc, float score) {
-      bottom.doc = docBase + doc;
-      bottom.score = score;
-      bottom =  pq.updateTop();
+    protected final void setBottom(int slot) {
+      for (LeafFieldComparator comparator : comparators) {
+        comparator.setBottom(slot);
+      }
     }
 
-    @Override
-    public void collect(int doc) throws IOException {
-      final float score = scorer.score();
-      if (score > maxScore) {
-        maxScore = score;
+    protected final int compareTop(int doc) throws IOException {
+      int cmp = firstReverseMul * firstComparator.compareTop(doc);
+      if (cmp != 0) {
+        return cmp;
       }
-      ++totalHits;
-      if (queueFull) {
-        if ((reverseMul * comparator.compareBottom(doc)) <= 0) {
-          // since docs are visited in doc Id order, if compare is 0, it means
-          // this document is largest than anything else in the queue, and
-          // therefore not competitive.
-          return;
-        }
-        
-        // This hit is competitive - replace bottom element in queue & adjustTop
-        comparator.copy(bottom.slot, doc);
-        updateBottom(doc, score);
-        comparator.setBottom(bottom.slot);
-      } else {
-        // Startup transient: queue hasn't gathered numHits yet
-        final int slot = totalHits - 1;
-        // Copy hit into queue
-        comparator.copy(slot, doc);
-        add(slot, doc, score);
-        if (queueFull) {
-          comparator.setBottom(bottom.slot);
+      for (int i = 1; i < comparators.length; ++i) {
+        cmp = reverseMul[i] * comparators[i].compareTop(doc);
+        if (cmp != 0) {
+          return cmp;
         }
       }
-
+      return 0;
     }
-    
+
     @Override
     public void setScorer(Scorer scorer) throws IOException {
       this.scorer = scorer;
-      super.setScorer(scorer);
-    }
-  }
-
-  /*
-   * Implements a TopFieldCollector over one SortField criteria, with tracking
-   * document scores and maxScore, and assumes out of orderness in doc Ids
-   * collection.
-   */
-  private static class OutOfOrderOneComparatorScoringMaxScoreCollector extends
-      OneComparatorScoringMaxScoreCollector {
-
-    public OutOfOrderOneComparatorScoringMaxScoreCollector(FieldValueHitQueue<Entry> queue,
-        int numHits, boolean fillFields) {
-      super(queue, numHits, fillFields);
-    }
-    
-    @Override
-    public void collect(int doc) throws IOException {
-      final float score = scorer.score();
-      if (score > maxScore) {
-        maxScore = score;
-      }
-      ++totalHits;
-      if (queueFull) {
-        // Fastmatch: return if this hit is not competitive
-        final int cmp = reverseMul * comparator.compareBottom(doc);
-        if (cmp < 0 || (cmp == 0 && doc + docBase > bottom.doc)) {
-          return;
-        }
-        
-        // This hit is competitive - replace bottom element in queue & adjustTop
-        comparator.copy(bottom.slot, doc);
-        updateBottom(doc, score);
-        comparator.setBottom(bottom.slot);
-      } else {
-        // Startup transient: queue hasn't gathered numHits yet
-        final int slot = totalHits - 1;
-        // Copy hit into queue
-        comparator.copy(slot, doc);
-        add(slot, doc, score);
-        if (queueFull) {
-          comparator.setBottom(bottom.slot);
-        }
+      for (LeafFieldComparator comparator : comparators) {
+        comparator.setScorer(scorer);
       }
     }
-    
+
     @Override
     public boolean acceptsDocsOutOfOrder() {
-      return true;
+      return false;
     }
-
   }
 
   /*
-   * Implements a TopFieldCollector over multiple SortField criteria, without
+   * Implements a TopFieldCollector over one SortField criteria, without
    * tracking document scores and maxScore.
    */
-  private static class MultiComparatorNonScoringCollector extends TopFieldCollector {
-    
-    final FieldComparator<?>[] comparators;
-    final int[] reverseMul;
+  private static class NonScoringCollector extends TopFieldCollector {
+
     final FieldValueHitQueue<Entry> queue;
-    public MultiComparatorNonScoringCollector(FieldValueHitQueue<Entry> queue,
-        int numHits, boolean fillFields) {
+
+    public NonScoringCollector(FieldValueHitQueue<Entry> queue, int numHits, boolean fillFields) {
       super(queue, numHits, fillFields);
       this.queue = queue;
-      comparators = queue.getComparators();
-      reverseMul = queue.getReverseMul();
-    }
-    
-    final void updateBottom(int doc) {
-      // bottom.score is already set to Float.NaN in add().
-      bottom.doc = docBase + doc;
-      bottom = pq.updateTop();
     }
 
     @Override
-    public void collect(int doc) throws IOException {
-      ++totalHits;
-      if (queueFull) {
-        // Fastmatch: return if this hit is not competitive
-        for (int i = 0;; i++) {
-          final int c = reverseMul[i] * comparators[i].compareBottom(doc);
-          if (c < 0) {
-            // Definitely not competitive.
-            return;
-          } else if (c > 0) {
-            // Definitely competitive.
-            break;
-          } else if (i == comparators.length - 1) {
-            // Here c=0. If we're at the last comparator, this doc is not
-            // competitive, since docs are visited in doc Id order, which means
-            // this doc cannot compete with any other document in the queue.
-            return;
-          }
-        }
+    public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+      docBase = context.docBase;
 
-        // This hit is competitive - replace bottom element in queue & adjustTop
-        for (int i = 0; i < comparators.length; i++) {
-          comparators[i].copy(bottom.slot, doc);
-        }
+      final LeafFieldComparator[] comparators = queue.getComparators(context);
+      final int[] reverseMul = queue.getReverseMul();
 
-        updateBottom(doc);
+      if (comparators.length == 1) {
+        return new OneComparatorLeafCollector(comparators[0], reverseMul[0]) {
 
-        for (int i = 0; i < comparators.length; i++) {
-          comparators[i].setBottom(bottom.slot);
-        }
+          @Override
+          public void collect(int doc) throws IOException {
+            ++totalHits;
+            if (queueFull) {
+              if ((reverseMul * comparator.compareBottom(doc)) <= 0) {
+                // since docs are visited in doc Id order, if compare is 0, it means
+                // this document is larger than anything else in the queue, and
+                // therefore not competitive.
+                return;
+              }
+
+              // This hit is competitive - replace bottom element in queue & adjustTop
+              comparator.copy(bottom.slot, doc);
+              updateBottom(doc);
+              comparator.setBottom(bottom.slot);
+            } else {
+              // Startup transient: queue hasn't gathered numHits yet
+              final int slot = totalHits - 1;
+              // Copy hit into queue
+              comparator.copy(slot, doc);
+              add(slot, doc, Float.NaN);
+              if (queueFull) {
+                comparator.setBottom(bottom.slot);
+              }
+            }
+          }
+
+        };
       } else {
-        // Startup transient: queue hasn't gathered numHits yet
-        final int slot = totalHits - 1;
-        // Copy hit into queue
-        for (int i = 0; i < comparators.length; i++) {
-          comparators[i].copy(slot, doc);
-        }
-        add(slot, doc, Float.NaN);
-        if (queueFull) {
-          for (int i = 0; i < comparators.length; i++) {
-            comparators[i].setBottom(bottom.slot);
+        return new MultiComparatorLeafCollector(comparators, reverseMul) {
+
+          @Override
+          public void collect(int doc) throws IOException {
+            ++totalHits;
+            if (queueFull) {
+              if ((compareBottom(doc)) <= 0) {
+                // since docs are visited in doc Id order, if compare is 0, it means
+                // this document is larger than anything else in the queue, and
+                // therefore not competitive.
+                return;
+              }
+
+              // This hit is competitive - replace bottom element in queue & adjustTop
+              copy(bottom.slot, doc);
+              updateBottom(doc);
+              setBottom(bottom.slot);
+            } else {
+              // Startup transient: queue hasn't gathered numHits yet
+              final int slot = totalHits - 1;
+              // Copy hit into queue
+              copy(slot, doc);
+              add(slot, doc, Float.NaN);
+              if (queueFull) {
+                setBottom(bottom.slot);
+              }
+            }
           }
-        }
-      }
-    }
 
-    @Override
-    protected void doSetNextReader(LeafReaderContext context) throws IOException {
-      docBase = context.docBase;
-      for (int i = 0; i < comparators.length; i++) {
-        queue.setComparator(i, comparators[i].setNextReader(context));
+        };
       }
     }
 
-    @Override
-    public void setScorer(Scorer scorer) throws IOException {
-      // set the scorer on all comparators
-      for (int i = 0; i < comparators.length; i++) {
-        comparators[i].setScorer(scorer);
-      }
-    }
   }
-  
+
   /*
-   * Implements a TopFieldCollector over multiple SortField criteria, without
+   * Implements a TopFieldCollector over one SortField criteria, without
    * tracking document scores and maxScore, and assumes out of orderness in doc
    * Ids collection.
    */
-  private static class OutOfOrderMultiComparatorNonScoringCollector extends
-      MultiComparatorNonScoringCollector {
-    
-    public OutOfOrderMultiComparatorNonScoringCollector(FieldValueHitQueue<Entry> queue,
-        int numHits, boolean fillFields) {
+  private static class OutOfOrderNonScoringCollector extends TopFieldCollector {
+
+    final FieldValueHitQueue<Entry> queue;
+
+    public OutOfOrderNonScoringCollector(FieldValueHitQueue<Entry> queue, int numHits, boolean fillFields) {
       super(queue, numHits, fillFields);
+      this.queue = queue;
     }
-    
+
     @Override
-    public void collect(int doc) throws IOException {
-      ++totalHits;
-      if (queueFull) {
-        // Fastmatch: return if this hit is not competitive
-        for (int i = 0;; i++) {
-          final int c = reverseMul[i] * comparators[i].compareBottom(doc);
-          if (c < 0) {
-            // Definitely not competitive.
-            return;
-          } else if (c > 0) {
-            // Definitely competitive.
-            break;
-          } else if (i == comparators.length - 1) {
-            // This is the equals case.
-            if (doc + docBase > bottom.doc) {
-              // Definitely not competitive
-              return;
+    public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+      docBase = context.docBase;
+
+      final LeafFieldComparator[] comparators = queue.getComparators(context);
+      final int[] reverseMul = queue.getReverseMul();
+
+      if (comparators.length == 1) {
+        return new OneComparatorLeafCollector(comparators[0], reverseMul[0]) {
+
+          @Override
+          public void collect(int doc) throws IOException {
+            ++totalHits;
+            if (queueFull) {
+              // Fastmatch: return if this hit is not competitive
+              final int cmp = reverseMul * comparator.compareBottom(doc);
+              if (cmp < 0 || (cmp == 0 && doc + docBase > bottom.doc)) {
+                return;
+              }
+
+              // This hit is competitive - replace bottom element in queue & adjustTop
+              comparator.copy(bottom.slot, doc);
+              updateBottom(doc);
+              comparator.setBottom(bottom.slot);
+            } else {
+              // Startup transient: queue hasn't gathered numHits yet
+              final int slot = totalHits - 1;
+              // Copy hit into queue
+              comparator.copy(slot, doc);
+              add(slot, doc, Float.NaN);
+              if (queueFull) {
+                comparator.setBottom(bottom.slot);
+              }
             }
-            break;
           }
-        }
 
-        // This hit is competitive - replace bottom element in queue & adjustTop
-        for (int i = 0; i < comparators.length; i++) {
-          comparators[i].copy(bottom.slot, doc);
-        }
-
-        updateBottom(doc);
+          @Override
+          public boolean acceptsDocsOutOfOrder() {
+            return true;
+          }
+
+        };
+      } else {
+        return new MultiComparatorLeafCollector(comparators, reverseMul) {
+
+          @Override
+          public void collect(int doc) throws IOException {
+            ++totalHits;
+            if (queueFull) {
+              // Fastmatch: return if this hit is not competitive
+              final int cmp = compareBottom(doc);
+              if (cmp < 0 || (cmp == 0 && doc + docBase > bottom.doc)) {
+                return;
+              }
+
+              // This hit is competitive - replace bottom element in queue & adjustTop
+              copy(bottom.slot, doc);
+              updateBottom(doc);
+              setBottom(bottom.slot);
+            } else {
+              // Startup transient: queue hasn't gathered numHits yet
+              final int slot = totalHits - 1;
+              // Copy hit into queue
+              copy(slot, doc);
+              add(slot, doc, Float.NaN);
+              if (queueFull) {
+                setBottom(bottom.slot);
+              }
+            }
+          }
 
-        for (int i = 0; i < comparators.length; i++) {
-          comparators[i].setBottom(bottom.slot);
-        }
-      } else {
-        // Startup transient: queue hasn't gathered numHits yet
-        final int slot = totalHits - 1;
-        // Copy hit into queue
-        for (int i = 0; i < comparators.length; i++) {
-          comparators[i].copy(slot, doc);
-        }
-        add(slot, doc, Float.NaN);
-        if (queueFull) {
-          for (int i = 0; i < comparators.length; i++) {
-            comparators[i].setBottom(bottom.slot);
+          @Override
+          public boolean acceptsDocsOutOfOrder() {
+            return true;
           }
-        }
+
+        };
       }
     }
-    
-    @Override
-    public boolean acceptsDocsOutOfOrder() {
-      return true;
-    }
 
   }
 
   /*
-   * Implements a TopFieldCollector over multiple SortField criteria, with
-   * tracking document scores and maxScore.
+   * Implements a TopFieldCollector over one SortField criteria, while tracking
+   * document scores but no maxScore.
    */
-  private static class MultiComparatorScoringMaxScoreCollector extends MultiComparatorNonScoringCollector {
-    
-    Scorer scorer;
-    
-    public MultiComparatorScoringMaxScoreCollector(FieldValueHitQueue<Entry> queue,
-        int numHits, boolean fillFields) {
+  private static class ScoringNoMaxScoreCollector extends TopFieldCollector {
+
+    final FieldValueHitQueue<Entry> queue;
+
+    public ScoringNoMaxScoreCollector(FieldValueHitQueue<Entry> queue, int numHits, boolean fillFields) {
       super(queue, numHits, fillFields);
-      // Must set maxScore to NEG_INF, or otherwise Math.max always returns NaN.
-      maxScore = Float.NEGATIVE_INFINITY;
-    }
-    
-    final void updateBottom(int doc, float score) {
-      bottom.doc = docBase + doc;
-      bottom.score = score;
-      bottom =  pq.updateTop();
+      this.queue = queue;
     }
 
     @Override
-    public void collect(int doc) throws IOException {
-      final float score = scorer.score();
-      if (score > maxScore) {
-        maxScore = score;
-      }
-      ++totalHits;
-      if (queueFull) {
-        // Fastmatch: return if this hit is not competitive
-        for (int i = 0;; i++) {
-          final int c = reverseMul[i] * comparators[i].compareBottom(doc);
-          if (c < 0) {
-            // Definitely not competitive.
-            return;
-          } else if (c > 0) {
-            // Definitely competitive.
-            break;
-          } else if (i == comparators.length - 1) {
-            // Here c=0. If we're at the last comparator, this doc is not
-            // competitive, since docs are visited in doc Id order, which means
-            // this doc cannot compete with any other document in the queue.
-            return;
-          }
-        }
+    public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+      docBase = context.docBase;
 
-        // This hit is competitive - replace bottom element in queue & adjustTop
-        for (int i = 0; i < comparators.length; i++) {
-          comparators[i].copy(bottom.slot, doc);
-        }
+      final LeafFieldComparator[] comparators = queue.getComparators(context);
+      final int[] reverseMul = queue.getReverseMul();
 
-        updateBottom(doc, score);
+      if (comparators.length == 1) {
+        return new OneComparatorLeafCollector(comparators[0], reverseMul[0]) {
 
-        for (int i = 0; i < comparators.length; i++) {
-          comparators[i].setBottom(bottom.slot);
-        }
+          @Override
+          public void collect(int doc) throws IOException {
+            ++totalHits;
+            if (queueFull) {
+              if ((reverseMul * comparator.compareBottom(doc)) <= 0) {
+                // since docs are visited in doc Id order, if compare is 0, it means
+                // this document is largest than anything else in the queue, and
+                // therefore not competitive.
+                return;
+              }
+
+              // Compute the score only if the hit is competitive.
+              final float score = scorer.score();
+
+              // This hit is competitive - replace bottom element in queue & adjustTop
+              comparator.copy(bottom.slot, doc);
+              updateBottom(doc, score);
+              comparator.setBottom(bottom.slot);
+            } else {
+              // Compute the score only if the hit is competitive.
+              final float score = scorer.score();
+
+              // Startup transient: queue hasn't gathered numHits yet
+              final int slot = totalHits - 1;
+              // Copy hit into queue
+              comparator.copy(slot, doc);
+              add(slot, doc, score);
+              if (queueFull) {
+                comparator.setBottom(bottom.slot);
+              }
+            }
+          }
+
+        };
       } else {
-        // Startup transient: queue hasn't gathered numHits yet
-        final int slot = totalHits - 1;
-        // Copy hit into queue
-        for (int i = 0; i < comparators.length; i++) {
-          comparators[i].copy(slot, doc);
-        }
-        add(slot, doc, score);
-        if (queueFull) {
-          for (int i = 0; i < comparators.length; i++) {
-            comparators[i].setBottom(bottom.slot);
+        return new MultiComparatorLeafCollector(comparators, reverseMul) {
+
+          @Override
+          public void collect(int doc) throws IOException {
+            ++totalHits;
+            if (queueFull) {
+              if ((compareBottom(doc)) <= 0) {
+                // since docs are visited in doc Id order, if compare is 0, it means
+                // this document is largest than anything else in the queue, and
+                // therefore not competitive.
+                return;
+              }
+
+              // Compute the score only if the hit is competitive.
+              final float score = scorer.score();
+
+              // This hit is competitive - replace bottom element in queue & adjustTop
+              copy(bottom.slot, doc);
+              updateBottom(doc, score);
+              setBottom(bottom.slot);
+            } else {
+              // Compute the score only if the hit is competitive.
+              final float score = scorer.score();
+
+              // Startup transient: queue hasn't gathered numHits yet
+              final int slot = totalHits - 1;
+              // Copy hit into queue
+              copy(slot, doc);
+              add(slot, doc, score);
+              if (queueFull) {
+                setBottom(bottom.slot);
+              }
+            }
           }
-        }
+
+        };
       }
     }
 
-    @Override
-    public void setScorer(Scorer scorer) throws IOException {
-      this.scorer = scorer;
-      super.setScorer(scorer);
-    }
   }
 
   /*
-   * Implements a TopFieldCollector over multiple SortField criteria, with
-   * tracking document scores and maxScore, and assumes out of orderness in doc
-   * Ids collection.
+   * Implements a TopFieldCollector over one SortField criteria, while tracking
+   * document scores but no maxScore, and assumes out of orderness in doc Ids
+   * collection.
    */
-  private final static class OutOfOrderMultiComparatorScoringMaxScoreCollector
-      extends MultiComparatorScoringMaxScoreCollector {
-    
-    public OutOfOrderMultiComparatorScoringMaxScoreCollector(FieldValueHitQueue<Entry> queue,
-        int numHits, boolean fillFields) {
+  private static class OutOfOrderScoringNoMaxScoreCollector extends TopFieldCollector {
+
+    final FieldValueHitQueue<Entry> queue;
+
+    public OutOfOrderScoringNoMaxScoreCollector(FieldValueHitQueue<Entry> queue, int numHits, boolean fillFields) {
       super(queue, numHits, fillFields);
+      this.queue = queue;
     }
-    
+
     @Override
-    public void collect(int doc) throws IOException {
-      final float score = scorer.score();
-      if (score > maxScore) {
-        maxScore = score;
-      }
-      ++totalHits;
-      if (queueFull) {
-        // Fastmatch: return if this hit is not competitive
-        for (int i = 0;; i++) {
-          final int c = reverseMul[i] * comparators[i].compareBottom(doc);
-          if (c < 0) {
-            // Definitely not competitive.
-            return;
-          } else if (c > 0) {
-            // Definitely competitive.
-            break;
-          } else if (i == comparators.length - 1) {
-            // This is the equals case.
-            if (doc + docBase > bottom.doc) {
-              // Definitely not competitive
-              return;
+    public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+      docBase = context.docBase;
+
+      final LeafFieldComparator[] comparators = queue.getComparators(context);
+      final int[] reverseMul = queue.getReverseMul();
+
+      if (comparators.length == 1) {
+        return new OneComparatorLeafCollector(comparators[0], reverseMul[0]) {
+
+          @Override
+          public void collect(int doc) throws IOException {
+            ++totalHits;
+            if (queueFull) {
+              // Fastmatch: return if this hit is not competitive
+              final int cmp = reverseMul * comparator.compareBottom(doc);
+              if (cmp < 0 || (cmp == 0 && doc + docBase > bottom.doc)) {
+                return;
+              }
+
+              // Compute the score only if the hit is competitive.
+              final float score = scorer.score();
+
+              // This hit is competitive - replace bottom element in queue & adjustTop
+              comparator.copy(bottom.slot, doc);
+              updateBottom(doc, score);
+              comparator.setBottom(bottom.slot);
+            } else {
+              // Compute the score only if the hit is competitive.
+              final float score = scorer.score();
+
+              // Startup transient: queue hasn't gathered numHits yet
+              final int slot = totalHits - 1;
+              // Copy hit into queue
+              comparator.copy(slot, doc);
+              add(slot, doc, score);
+              if (queueFull) {
+                comparator.setBottom(bottom.slot);
+              }
             }
-            break;
           }
-        }
 
-        // This hit is competitive - replace bottom element in queue & adjustTop
-        for (int i = 0; i < comparators.length; i++) {
-          comparators[i].copy(bottom.slot, doc);
-        }
-
-        updateBottom(doc, score);
+          @Override
+          public boolean acceptsDocsOutOfOrder() {
+            return true;
+          }
+
+        };
+      } else {
+        return new MultiComparatorLeafCollector(comparators, reverseMul) {
+
+          @Override
+          public void collect(int doc) throws IOException {
+            ++totalHits;
+            if (queueFull) {
+              // Fastmatch: return if this hit is not competitive
+              final int cmp = compareBottom(doc);
+              if (cmp < 0 || (cmp == 0 && doc + docBase > bottom.doc)) {
+                return;
+              }
+
+              // Compute the score only if the hit is competitive.
+              final float score = scorer.score();
+
+              // This hit is competitive - replace bottom element in queue & adjustTop
+              copy(bottom.slot, doc);
+              updateBottom(doc, score);
+              setBottom(bottom.slot);
+            } else {
+              // Compute the score only if the hit is competitive.
+              final float score = scorer.score();
+
+              // Startup transient: queue hasn't gathered numHits yet
+              final int slot = totalHits - 1;
+              // Copy hit into queue
+              copy(slot, doc);
+              add(slot, doc, score);
+              if (queueFull) {
+                setBottom(bottom.slot);
+              }
+            }
+          }
 
-        for (int i = 0; i < comparators.length; i++) {
-          comparators[i].setBottom(bottom.slot);
-        }
-      } else {
-        // Startup transient: queue hasn't gathered numHits yet
-        final int slot = totalHits - 1;
-        // Copy hit into queue
-        for (int i = 0; i < comparators.length; i++) {
-          comparators[i].copy(slot, doc);
-        }
-        add(slot, doc, score);
-        if (queueFull) {
-          for (int i = 0; i < comparators.length; i++) {
-            comparators[i].setBottom(bottom.slot);
+          @Override
+          public boolean acceptsDocsOutOfOrder() {
+            return true;
           }
-        }
+        };
       }
     }
-    
-    @Override
-    public boolean acceptsDocsOutOfOrder() {
-      return true;
-    }
 
   }
 
   /*
-   * Implements a TopFieldCollector over multiple SortField criteria, with
-   * tracking document scores and maxScore.
+   * Implements a TopFieldCollector over one SortField criteria, with tracking
+   * document scores and maxScore.
    */
-  private static class MultiComparatorScoringNoMaxScoreCollector extends MultiComparatorNonScoringCollector {
-    
-    Scorer scorer;
-    
-    public MultiComparatorScoringNoMaxScoreCollector(FieldValueHitQueue<Entry> queue,
-        int numHits, boolean fillFields) {
+  private static class ScoringMaxScoreCollector extends TopFieldCollector {
+
+    final FieldValueHitQueue<Entry> queue;
+
+    public ScoringMaxScoreCollector(FieldValueHitQueue<Entry> queue, int numHits, boolean fillFields) {
       super(queue, numHits, fillFields);
-    }
-    
-    final void updateBottom(int doc, float score) {
-      bottom.doc = docBase + doc;
-      bottom.score = score;
-      bottom = pq.updateTop();
+      this.queue = queue;
+      maxScore = Float.MIN_NORMAL; // otherwise we would keep NaN
     }
 
     @Override
-    public void collect(int doc) throws IOException {
-      ++totalHits;
-      if (queueFull) {
-        // Fastmatch: return if this hit is not competitive
-        for (int i = 0;; i++) {
-          final int c = reverseMul[i] * comparators[i].compareBottom(doc);
-          if (c < 0) {
-            // Definitely not competitive.
-            return;
-          } else if (c > 0) {
-            // Definitely competitive.
-            break;
-          } else if (i == comparators.length - 1) {
-            // Here c=0. If we're at the last comparator, this doc is not
-            // competitive, since docs are visited in doc Id order, which means
-            // this doc cannot compete with any other document in the queue.
-            return;
-          }
-        }
+    public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+      docBase = context.docBase;
 
-        // This hit is competitive - replace bottom element in queue & adjustTop
-        for (int i = 0; i < comparators.length; i++) {
-          comparators[i].copy(bottom.slot, doc);
-        }
+      final LeafFieldComparator[] comparators = queue.getComparators(context);
+      final int[] reverseMul = queue.getReverseMul();
 
-        // Compute score only if it is competitive.
-        final float score = scorer.score();
-        updateBottom(doc, score);
+      if (comparators.length == 1) {
+        return new OneComparatorLeafCollector(comparators[0], reverseMul[0]) {
 
-        for (int i = 0; i < comparators.length; i++) {
-          comparators[i].setBottom(bottom.slot);
-        }
+          @Override
+          public void collect(int doc) throws IOException {
+            final float score = scorer.score();
+            if (score > maxScore) {
+              maxScore = score;
+            }
+            ++totalHits;
+            if (queueFull) {
+              if (reverseMul * comparator.compareBottom(doc) <= 0) {
+                // since docs are visited in doc Id order, if compare is 0, it means
+                // this document is largest than anything else in the queue, and
+                // therefore not competitive.
+                return;
+              }
+
+              // This hit is competitive - replace bottom element in queue & adjustTop
+              comparator.copy(bottom.slot, doc);
+              updateBottom(doc, score);
+              comparator.setBottom(bottom.slot);
+            } else {
+              // Startup transient: queue hasn't gathered numHits yet
+              final int slot = totalHits - 1;
+              // Copy hit into queue
+              comparator.copy(slot, doc);
+              add(slot, doc, score);
+              if (queueFull) {
+                comparator.setBottom(bottom.slot);
+              }
+            }
+          }
+
+        };
       } else {
-        // Startup transient: queue hasn't gathered numHits yet
-        final int slot = totalHits - 1;
-        // Copy hit into queue
-        for (int i = 0; i < comparators.length; i++) {
-          comparators[i].copy(slot, doc);
-        }
+        return new MultiComparatorLeafCollector(comparators, reverseMul) {
 
-        // Compute score only if it is competitive.
-        final float score = scorer.score();
-        add(slot, doc, score);
-        if (queueFull) {
-          for (int i = 0; i < comparators.length; i++) {
-            comparators[i].setBottom(bottom.slot);
+          @Override
+          public void collect(int doc) throws IOException {
+            final float score = scorer.score();
+            if (score > maxScore) {
+              maxScore = score;
+            }
+            ++totalHits;
+            if (queueFull) {
+              if (compareBottom(doc) <= 0) {
+                // since docs are visited in doc Id order, if compare is 0, it means
+                // this document is largest than anything else in the queue, and
+                // therefore not competitive.
+                return;
+              }
+
+              // This hit is competitive - replace bottom element in queue & adjustTop
+              copy(bottom.slot, doc);
+              updateBottom(doc, score);
+              setBottom(bottom.slot);
+            } else {
+              // Startup transient: queue hasn't gathered numHits yet
+              final int slot = totalHits - 1;
+              // Copy hit into queue
+              copy(slot, doc);
+              add(slot, doc, score);
+              if (queueFull) {
+                setBottom(bottom.slot);
+              }
+            }
           }
-        }
+
+        };
       }
     }
 
-    @Override
-    public void setScorer(Scorer scorer) throws IOException {
-      this.scorer = scorer;
-      super.setScorer(scorer);
-    }
   }
 
   /*
-   * Implements a TopFieldCollector over multiple SortField criteria, with
-   * tracking document scores and maxScore, and assumes out of orderness in doc
-   * Ids collection.
+   * Implements a TopFieldCollector over one SortField criteria, with tracking
+   * document scores and maxScore, and assumes out of orderness in doc Ids
+   * collection.
    */
-  private final static class OutOfOrderMultiComparatorScoringNoMaxScoreCollector
-      extends MultiComparatorScoringNoMaxScoreCollector {
-    
-    public OutOfOrderMultiComparatorScoringNoMaxScoreCollector(
-        FieldValueHitQueue<Entry> queue, int numHits, boolean fillFields) {
+  private static class OutOfOrderScoringMaxScoreCollector extends TopFieldCollector {
+
+    final FieldValueHitQueue<Entry> queue;
+
+    public OutOfOrderScoringMaxScoreCollector(FieldValueHitQueue<Entry> queue, int numHits, boolean fillFields) {
       super(queue, numHits, fillFields);
+      this.queue = queue;
+      maxScore = Float.MIN_NORMAL; // otherwise we would keep NaN
     }
-    
+
     @Override
-    public void collect(int doc) throws IOException {
-      ++totalHits;
-      if (queueFull) {
-        // Fastmatch: return if this hit is not competitive
-        for (int i = 0;; i++) {
-          final int c = reverseMul[i] * comparators[i].compareBottom(doc);
-          if (c < 0) {
-            // Definitely not competitive.
-            return;
-          } else if (c > 0) {
-            // Definitely competitive.
-            break;
-          } else if (i == comparators.length - 1) {
-            // This is the equals case.
-            if (doc + docBase > bottom.doc) {
-              // Definitely not competitive
-              return;
-            }
-            break;
-          }
-        }
+    public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+      docBase = context.docBase;
 
-        // This hit is competitive - replace bottom element in queue & adjustTop
-        for (int i = 0; i < comparators.length; i++) {
-          comparators[i].copy(bottom.slot, doc);
-        }
+      final LeafFieldComparator[] comparators = queue.getComparators(context);
+      final int[] reverseMul = queue.getReverseMul();
 
-        // Compute score only if it is competitive.
-        final float score = scorer.score();
-        updateBottom(doc, score);
+      if (comparators.length == 1) {
+        return new OneComparatorLeafCollector(comparators[0], reverseMul[0]) {
 
-        for (int i = 0; i < comparators.length; i++) {
-          comparators[i].setBottom(bottom.slot);
-        }
-      } else {
-        // Startup transient: queue hasn't gathered numHits yet
-        final int slot = totalHits - 1;
-        // Copy hit into queue
-        for (int i = 0; i < comparators.length; i++) {
-          comparators[i].copy(slot, doc);
-        }
+          @Override
+          public void collect(int doc) throws IOException {
+            final float score = scorer.score();
+            if (score > maxScore) {
+              maxScore = score;
+            }
+            ++totalHits;
+            if (queueFull) {
+              // Fastmatch: return if this hit is not competitive
+              final int cmp = reverseMul * comparator.compareBottom(doc);
+              if (cmp < 0 || (cmp == 0 && doc + docBase > bottom.doc)) {
+                return;
+              }
+
+              // This hit is competitive - replace bottom element in queue & adjustTop
+              comparator.copy(bottom.slot, doc);
+              updateBottom(doc, score);
+              comparator.setBottom(bottom.slot);
+            } else {
+              // Startup transient: queue hasn't gathered numHits yet
+              final int slot = totalHits - 1;
+              // Copy hit into queue
+              comparator.copy(slot, doc);
+              add(slot, doc, score);
+              if (queueFull) {
+                comparator.setBottom(bottom.slot);
+              }
+            }
+          }
 
-        // Compute score only if it is competitive.
-        final float score = scorer.score();
-        add(slot, doc, score);
-        if (queueFull) {
-          for (int i = 0; i < comparators.length; i++) {
-            comparators[i].setBottom(bottom.slot);
+          @Override
+          public boolean acceptsDocsOutOfOrder() {
+            return true;
+          }
+        };
+      } else {
+        return new MultiComparatorLeafCollector(comparators, reverseMul) {
+
+          @Override
+          public void collect(int doc) throws IOException {
+            final float score = scorer.score();
+            if (score > maxScore) {
+              maxScore = score;
+            }
+            ++totalHits;
+            if (queueFull) {
+              // Fastmatch: return if this hit is not competitive
+              final int cmp = compareBottom(doc);
+              if (cmp < 0 || (cmp == 0 && doc + docBase > bottom.doc)) {
+                return;
+              }
+
+              // This hit is competitive - replace bottom element in queue & adjustTop
+              copy(bottom.slot, doc);
+              updateBottom(doc, score);
+              setBottom(bottom.slot);
+            } else {
+              // Startup transient: queue hasn't gathered numHits yet
+              final int slot = totalHits - 1;
+              // Copy hit into queue
+              copy(slot, doc);
+              add(slot, doc, score);
+              if (queueFull) {
+                setBottom(bottom.slot);
+              }
+            }
           }
-        }
-      }
-    }
 
-    @Override
-    public void setScorer(Scorer scorer) throws IOException {
-      this.scorer = scorer;
-      super.setScorer(scorer);
-    }
-    
-    @Override
-    public boolean acceptsDocsOutOfOrder() {
-      return true;
+          @Override
+          public boolean acceptsDocsOutOfOrder() {
+            return true;
+          }
+        };
+      }
     }
 
   }
@@ -846,30 +724,24 @@ public abstract class TopFieldCollector
    */
   private final static class PagingFieldCollector extends TopFieldCollector {
 
-    Scorer scorer;
     int collectedHits;
-    final FieldComparator<?>[] comparators;
-    final int[] reverseMul;
     final FieldValueHitQueue<Entry> queue;
     final boolean trackDocScores;
     final boolean trackMaxScore;
     final FieldDoc after;
-    int afterDoc;
-    
-    public PagingFieldCollector(
-                                FieldValueHitQueue<Entry> queue, FieldDoc after, int numHits, boolean fillFields,
+
+    public PagingFieldCollector(FieldValueHitQueue<Entry> queue, FieldDoc after, int numHits, boolean fillFields,
                                 boolean trackDocScores, boolean trackMaxScore) {
       super(queue, numHits, fillFields);
       this.queue = queue;
       this.trackDocScores = trackDocScores;
       this.trackMaxScore = trackMaxScore;
       this.after = after;
-      comparators = queue.getComparators();
-      reverseMul = queue.getReverseMul();
 
       // Must set maxScore to NEG_INF, or otherwise Math.max always returns NaN.
       maxScore = Float.NEGATIVE_INFINITY;
 
+      FieldComparator<?>[] comparators = queue.comparators;
       // Tell all comparators their top value:
       for(int i=0;i<comparators.length;i++) {
         @SuppressWarnings("unchecked")
@@ -877,141 +749,86 @@ public abstract class TopFieldCollector
         comparator.setTopValue(after.fields[i]);
       }
     }
-    
-    void updateBottom(int doc, float score) {
-      bottom.doc = docBase + doc;
-      bottom.score = score;
-      bottom = pq.updateTop();
-    }
 
-    @SuppressWarnings({"unchecked", "rawtypes"})
     @Override
-    public void collect(int doc) throws IOException {
-      //System.out.println("  collect doc=" + doc);
+    public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+      docBase = context.docBase;
+      final int afterDoc = after.doc - docBase;
+      return new MultiComparatorLeafCollector(queue.getComparators(context), queue.getReverseMul()) {
 
-      totalHits++;
+        @Override
+        public void collect(int doc) throws IOException {
+          //System.out.println("  collect doc=" + doc);
 
-      float score = Float.NaN;
-      if (trackMaxScore) {
-        score = scorer.score();
-        if (score > maxScore) {
-          maxScore = score;
-        }
-      }
+          totalHits++;
 
-      if (queueFull) {
-        // Fastmatch: return if this hit is no better than
-        // the worst hit currently in the queue:
-        for (int i = 0;; i++) {
-          final int c = reverseMul[i] * comparators[i].compareBottom(doc);
-          if (c < 0) {
-            // Definitely not competitive.
-            return;
-          } else if (c > 0) {
-            // Definitely competitive.
-            break;
-          } else if (i == comparators.length - 1) {
-            // This is the equals case.
-            if (doc + docBase > bottom.doc) {
-              // Definitely not competitive
-              return;
+          float score = Float.NaN;
+          if (trackMaxScore) {
+            score = scorer.score();
+            if (score > maxScore) {
+              maxScore = score;
             }
-            break;
           }
-        }
-      }
-
-      // Check if this hit was already collected on a
-      // previous page:
-      boolean sameValues = true;
-      for(int compIDX=0;compIDX<comparators.length;compIDX++) {
-        final FieldComparator comp = comparators[compIDX];
-
-        final int cmp = reverseMul[compIDX] * comp.compareTop(doc);
-        if (cmp > 0) {
-          // Already collected on a previous page
-          //System.out.println("    skip: before");
-          return;
-        } else if (cmp < 0) {
-          // Not yet collected
-          sameValues = false;
-          //System.out.println("    keep: after; reverseMul=" + reverseMul[compIDX]);
-          break;
-        }
-      }
-
-      // Tie-break by docID:
-      if (sameValues && doc <= afterDoc) {
-        // Already collected on a previous page
-        //System.out.println("    skip: tie-break");
-        return;
-      }
 
-      if (queueFull) {
-        // This hit is competitive - replace bottom element in queue & adjustTop
-        for (int i = 0; i < comparators.length; i++) {
-          comparators[i].copy(bottom.slot, doc);
-        }
+          if (queueFull) {
+            // Fastmatch: return if this hit is no better than
+            // the worst hit currently in the queue:
+            final int cmp = compareBottom(doc);
+            if (cmp < 0 || (cmp == 0 && doc + docBase > bottom.doc)) {
+              // Definitely not competitive.
+              return;
+            }
+          }
 
-        // Compute score only if it is competitive.
-        if (trackDocScores && !trackMaxScore) {
-          score = scorer.score();
-        }
-        updateBottom(doc, score);
+          final int topCmp = compareTop(doc);
+          if (topCmp > 0 || (topCmp == 0 && doc <= afterDoc)) {
+            // Already collected on a previous page
+            return;
+          }
 
-        for (int i = 0; i < comparators.length; i++) {
-          comparators[i].setBottom(bottom.slot);
-        }
-      } else {
-        collectedHits++;
+          if (queueFull) {
+            // This hit is competitive - replace bottom element in queue & adjustTop
+            copy(bottom.slot, doc);
+
+            // Compute score only if it is competitive.
+            if (trackDocScores && !trackMaxScore) {
+              score = scorer.score();
+            }
+            updateBottom(doc, score);
 
-        // Startup transient: queue hasn't gathered numHits yet
-        final int slot = collectedHits - 1;
-        //System.out.println("    slot=" + slot);
-        // Copy hit into queue
-        for (int i = 0; i < comparators.length; i++) {
-          comparators[i].copy(slot, doc);
-        }
+            setBottom(bottom.slot);
+          } else {
+            collectedHits++;
 
-        // Compute score only if it is competitive.
-        if (trackDocScores && !trackMaxScore) {
-          score = scorer.score();
-        }
-        bottom = pq.add(new Entry(slot, docBase + doc, score));
-        queueFull = collectedHits == numHits;
-        if (queueFull) {
-          for (int i = 0; i < comparators.length; i++) {
-            comparators[i].setBottom(bottom.slot);
+            // Startup transient: queue hasn't gathered numHits yet
+            final int slot = collectedHits - 1;
+            //System.out.println("    slot=" + slot);
+            // Copy hit into queue
+            copy(slot, doc);
+
+            // Compute score only if it is competitive.
+            if (trackDocScores && !trackMaxScore) {
+              score = scorer.score();
+            }
+            bottom = pq.add(new Entry(slot, docBase + doc, score));
+            queueFull = collectedHits == numHits;
+            if (queueFull) {
+              setBottom(bottom.slot);
+            }
           }
         }
-      }
-    }
 
-    @Override
-    public void setScorer(Scorer scorer) {
-      this.scorer = scorer;
-      for (int i = 0; i < comparators.length; i++) {
-        comparators[i].setScorer(scorer);
-      }
-    }
-    
-    @Override
-    public boolean acceptsDocsOutOfOrder() {
-      return true;
+        @Override
+        public boolean acceptsDocsOutOfOrder() {
+          return true;
+        }
+      };
     }
 
-    @Override
-    protected void doSetNextReader(LeafReaderContext context) throws IOException {
-      docBase = context.docBase;
-      afterDoc = after.doc - docBase;
-      for (int i = 0; i < comparators.length; i++) {
-        queue.setComparator(i, comparators[i].setNextReader(context));
-      }
-    }
   }
 
   private static final ScoreDoc[] EMPTY_SCOREDOCS = new ScoreDoc[0];
-  
+
   private final boolean fillFields;
 
   /*
@@ -1024,7 +841,7 @@ public abstract class TopFieldCollector
   FieldValueHitQueue.Entry bottom = null;
   boolean queueFull;
   int docBase;
-  
+
   // Declaring the constructor private prevents extending this class by anyone
   // else. Note that the class cannot be final since it's extended by the
   // internal versions. If someone will define a constructor with any other
@@ -1043,7 +860,7 @@ public abstract class TopFieldCollector
    * <p><b>NOTE</b>: The instances returned by this method
    * pre-allocate a full array of length
    * <code>numHits</code>.
-   * 
+   *
    * @param sort
    *          the sort criteria (SortFields).
    * @param numHits
@@ -1067,7 +884,7 @@ public abstract class TopFieldCollector
    *          <code>trackDocScores</code> to true as well.
    * @param docsScoredInOrder
    *          specifies whether documents are scored in doc Id order or not by
-   *          the given {@link Scorer} in {@link #setScorer(Scorer)}.
+   *          the given {@link Scorer} in {@link LeafCollector#setScorer(Scorer)}.
    * @return a {@link TopFieldCollector} instance which will sort the results by
    *         the sort criteria.
    * @throws IOException if there is a low-level I/O error
@@ -1086,7 +903,7 @@ public abstract class TopFieldCollector
    * <p><b>NOTE</b>: The instances returned by this method
    * pre-allocate a full array of length
    * <code>numHits</code>.
-   * 
+   *
    * @param sort
    *          the sort criteria (SortFields).
    * @param numHits
@@ -1112,7 +929,7 @@ public abstract class TopFieldCollector
    *          <code>trackDocScores</code> to true as well.
    * @param docsScoredInOrder
    *          specifies whether documents are scored in doc Id order or not by
-   *          the given {@link Scorer} in {@link #setScorer(Scorer)}.
+   *          the given {@link Scorer} in {@link LeafCollector#setScorer(Scorer)}.
    * @return a {@link TopFieldCollector} instance which will sort the results by
    *         the sort criteria.
    * @throws IOException if there is a low-level I/O error
@@ -1125,7 +942,7 @@ public abstract class TopFieldCollector
     if (sort.fields.length == 0) {
       throw new IllegalArgumentException("Sort must contain at least one field");
     }
-    
+
     if (numHits <= 0) {
       throw new IllegalArgumentException("numHits must be > 0; please use TotalHitCountCollector if you just need the total hit count");
     }
@@ -1133,42 +950,21 @@ public abstract class TopFieldCollector
     FieldValueHitQueue<Entry> queue = FieldValueHitQueue.create(sort.fields, numHits);
 
     if (after == null) {
-      if (queue.getComparators().length == 1) {
-        if (docsScoredInOrder) {
-          if (trackMaxScore) {
-            return new OneComparatorScoringMaxScoreCollector(queue, numHits, fillFields);
-          } else if (trackDocScores) {
-            return new OneComparatorScoringNoMaxScoreCollector(queue, numHits, fillFields);
-          } else {
-            return new OneComparatorNonScoringCollector(queue, numHits, fillFields);
-          }
-        } else {
-          if (trackMaxScore) {
-            return new OutOfOrderOneComparatorScoringMaxScoreCollector(queue, numHits, fillFields);
-          } else if (trackDocScores) {
-            return new OutOfOrderOneComparatorScoringNoMaxScoreCollector(queue, numHits, fillFields);
-          } else {
-            return new OutOfOrderOneComparatorNonScoringCollector(queue, numHits, fillFields);
-          }
-        }
-      }
-
-      // multiple comparators.
       if (docsScoredInOrder) {
         if (trackMaxScore) {
-          return new MultiComparatorScoringMaxScoreCollector(queue, numHits, fillFields);
+          return new ScoringMaxScoreCollector(queue, numHits, fillFields);
         } else if (trackDocScores) {
-          return new MultiComparatorScoringNoMaxScoreCollector(queue, numHits, fillFields);
+          return new ScoringNoMaxScoreCollector(queue, numHits, fillFields);
         } else {
-          return new MultiComparatorNonScoringCollector(queue, numHits, fillFields);
+          return new NonScoringCollector(queue, numHits, fillFields);
         }
       } else {
         if (trackMaxScore) {
-          return new OutOfOrderMultiComparatorScoringMaxScoreCollector(queue, numHits, fillFields);
+          return new OutOfOrderScoringMaxScoreCollector(queue, numHits, fillFields);
         } else if (trackDocScores) {
-          return new OutOfOrderMultiComparatorScoringNoMaxScoreCollector(queue, numHits, fillFields);
+          return new OutOfOrderScoringNoMaxScoreCollector(queue, numHits, fillFields);
         } else {
-          return new OutOfOrderMultiComparatorNonScoringCollector(queue, numHits, fillFields);
+          return new OutOfOrderNonScoringCollector(queue, numHits, fillFields);
         }
       }
     } else {
@@ -1183,12 +979,24 @@ public abstract class TopFieldCollector
       return new PagingFieldCollector(queue, after, numHits, fillFields, trackDocScores, trackMaxScore);
     }
   }
-  
+
   final void add(int slot, int doc, float score) {
     bottom = pq.add(new Entry(slot, docBase + doc, score));
     queueFull = totalHits == numHits;
   }
 
+  final void updateBottom(int doc) {
+    // bottom.score is already set to Float.NaN in add().
+    bottom.doc = docBase + doc;
+    bottom = pq.updateTop();
+  }
+
+  final void updateBottom(int doc, float score) {
+    bottom.doc = docBase + doc;
+    bottom.score = score;
+    bottom = pq.updateTop();
+  }
+
   /*
    * Only the following callback methods need to be overridden since
    * topDocs(int, int) calls them to return the results.
@@ -1209,7 +1017,7 @@ public abstract class TopFieldCollector
       }
     }
   }
-  
+
   @Override
   protected TopDocs newTopDocs(ScoreDoc[] results, int start) {
     if (results == null) {
@@ -1218,12 +1026,8 @@ public abstract class TopFieldCollector
       maxScore = Float.NaN;
     }
 
-    // If this is a maxScoring tracking collector and there were no results, 
+    // If this is a maxScoring tracking collector and there were no results,
     return new TopFieldDocs(totalHits, results, ((FieldValueHitQueue<Entry>) pq).getFields(), maxScore);
   }
-  
-  @Override
-  public boolean acceptsDocsOutOfOrder() {
-    return false;
-  }
+
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java?rev=1649818&r1=1649817&r2=1649818&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java Tue Jan  6 15:03:19 2015
@@ -36,198 +36,189 @@ import org.apache.lucene.index.LeafReade
  */
 public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
 
-  // Assumes docs are scored in order.
-  private static class InOrderTopScoreDocCollector extends TopScoreDocCollector {
-    private InOrderTopScoreDocCollector(int numHits) {
-      super(numHits);
-    }
-    
-    @Override
-    public void collect(int doc) throws IOException {
-      float score = scorer.score();
+  private abstract static class ScorerLeafCollector implements LeafCollector {
 
-      // This collector cannot handle these scores:
-      assert score != Float.NEGATIVE_INFINITY;
-      assert !Float.isNaN(score);
-
-      totalHits++;
-      if (score <= pqTop.score) {
-        // Since docs are returned in-order (i.e., increasing doc Id), a document
-        // with equal score to pqTop.score cannot compete since HitQueue favors
-        // documents with lower doc Ids. Therefore reject those docs too.
-        return;
-      }
-      pqTop.doc = doc + docBase;
-      pqTop.score = score;
-      pqTop = pq.updateTop();
-    }
-    
-    @Override
-    public boolean acceptsDocsOutOfOrder() {
-      return false;
-    }
-  }
-  
-  // Assumes docs are scored in order.
-  private static class InOrderPagingScoreDocCollector extends TopScoreDocCollector {
-    private final ScoreDoc after;
-    // this is always after.doc - docBase, to save an add when score == after.score
-    private int afterDoc;
-    private int collectedHits;
+    final boolean scoreDocsInOrder;
 
-    private InOrderPagingScoreDocCollector(ScoreDoc after, int numHits) {
-      super(numHits);
-      this.after = after;
+    ScorerLeafCollector(boolean scoreDocsInOrder) {
+      this.scoreDocsInOrder = scoreDocsInOrder;
     }
-    
-    @Override
-    public void collect(int doc) throws IOException {
-      float score = scorer.score();
 
-      // This collector cannot handle these scores:
-      assert score != Float.NEGATIVE_INFINITY;
-      assert !Float.isNaN(score);
-
-      totalHits++;
-      
-      if (score > after.score || (score == after.score && doc <= afterDoc)) {
-        // hit was collected on a previous page
-        return;
-      }
-      
-      if (score <= pqTop.score) {
-        // Since docs are returned in-order (i.e., increasing doc Id), a document
-        // with equal score to pqTop.score cannot compete since HitQueue favors
-        // documents with lower doc Ids. Therefore reject those docs too.
-        return;
-      }
-      collectedHits++;
-      pqTop.doc = doc + docBase;
-      pqTop.score = score;
-      pqTop = pq.updateTop();
-    }
+    Scorer scorer;
 
     @Override
-    public boolean acceptsDocsOutOfOrder() {
-      return false;
+    public void setScorer(Scorer scorer) throws IOException {
+      this.scorer = scorer;
     }
 
     @Override
-    protected void doSetNextReader(LeafReaderContext context) throws IOException {
-      super.doSetNextReader(context);
-      afterDoc = after.doc - context.docBase;
+    public boolean acceptsDocsOutOfOrder() {
+      return scoreDocsInOrder == false;
     }
 
-    @Override
-    protected int topDocsSize() {
-      return collectedHits < pq.size() ? collectedHits : pq.size();
-    }
-    
-    @Override
-    protected TopDocs newTopDocs(ScoreDoc[] results, int start) {
-      return results == null ? new TopDocs(totalHits, new ScoreDoc[0], Float.NaN) : new TopDocs(totalHits, results);
-    }
   }
 
-  // Assumes docs are scored out of order.
-  private static class OutOfOrderTopScoreDocCollector extends TopScoreDocCollector {
-    private OutOfOrderTopScoreDocCollector(int numHits) {
+  private static class SimpleTopScoreDocCollector extends TopScoreDocCollector {
+
+    private final boolean scoreDocsInOrder;
+
+    SimpleTopScoreDocCollector(int numHits, boolean scoreDocsInOrder) {
       super(numHits);
+      this.scoreDocsInOrder = scoreDocsInOrder;
     }
-    
+
     @Override
-    public void collect(int doc) throws IOException {
-      float score = scorer.score();
+    public LeafCollector getLeafCollector(LeafReaderContext context)
+        throws IOException {
+      final int docBase = context.docBase;
+      if (scoreDocsInOrder) {
+        return new ScorerLeafCollector(scoreDocsInOrder) {
 
-      // This collector cannot handle NaN
-      assert !Float.isNaN(score);
+          @Override
+          public void collect(int doc) throws IOException {
+            float score = scorer.score();
 
-      totalHits++;
-      if (score < pqTop.score) {
-        // Doesn't compete w/ bottom entry in queue
-        return;
-      }
-      doc += docBase;
-      if (score == pqTop.score && doc > pqTop.doc) {
-        // Break tie in score by doc ID:
-        return;
+            // This collector cannot handle these scores:
+            assert score != Float.NEGATIVE_INFINITY;
+            assert !Float.isNaN(score);
+
+            totalHits++;
+            if (score <= pqTop.score) {
+              // Since docs are returned in-order (i.e., increasing doc Id), a document
+              // with equal score to pqTop.score cannot compete since HitQueue favors
+              // documents with lower doc Ids. Therefore reject those docs too.
+              return;
+            }
+            pqTop.doc = doc + docBase;
+            pqTop.score = score;
+            pqTop = pq.updateTop();
+          }
+
+        };
+      } else {
+        return new ScorerLeafCollector(scoreDocsInOrder) {
+
+          @Override
+          public void collect(int doc) throws IOException {
+            float score = scorer.score();
+
+            // This collector cannot handle NaN
+            assert !Float.isNaN(score);
+
+            totalHits++;
+            if (score < pqTop.score) {
+              // Doesn't compete w/ bottom entry in queue
+              return;
+            }
+            doc += docBase;
+            if (score == pqTop.score && doc > pqTop.doc) {
+              // Break tie in score by doc ID:
+              return;
+            }
+            pqTop.doc = doc;
+            pqTop.score = score;
+            pqTop = pq.updateTop();
+          }
+
+        };
       }
-      pqTop.doc = doc;
-      pqTop.score = score;
-      pqTop = pq.updateTop();
-    }
-    
-    @Override
-    public boolean acceptsDocsOutOfOrder() {
-      return true;
     }
+
   }
-  
-  // Assumes docs are scored out of order.
-  private static class OutOfOrderPagingScoreDocCollector extends TopScoreDocCollector {
+
+  private static class PagingTopScoreDocCollector extends TopScoreDocCollector {
+
+    private final boolean scoreDocsInOrder;
     private final ScoreDoc after;
-    // this is always after.doc - docBase, to save an add when score == after.score
-    private int afterDoc;
     private int collectedHits;
 
-    private OutOfOrderPagingScoreDocCollector(ScoreDoc after, int numHits) {
+    PagingTopScoreDocCollector(int numHits, boolean scoreDocsInOrder, ScoreDoc after) {
       super(numHits);
+      this.scoreDocsInOrder = scoreDocsInOrder;
       this.after = after;
+      this.collectedHits = 0;
     }
-    
-    @Override
-    public void collect(int doc) throws IOException {
-      float score = scorer.score();
 
-      // This collector cannot handle NaN
-      assert !Float.isNaN(score);
-
-      totalHits++;
-      if (score > after.score || (score == after.score && doc <= afterDoc)) {
-        // hit was collected on a previous page
-        return;
-      }
-      if (score < pqTop.score) {
-        // Doesn't compete w/ bottom entry in queue
-        return;
-      }
-      doc += docBase;
-      if (score == pqTop.score && doc > pqTop.doc) {
-        // Break tie in score by doc ID:
-        return;
-      }
-      collectedHits++;
-      pqTop.doc = doc;
-      pqTop.score = score;
-      pqTop = pq.updateTop();
-    }
-    
-    @Override
-    public boolean acceptsDocsOutOfOrder() {
-      return true;
-    }
-    
-    @Override
-    protected void doSetNextReader(LeafReaderContext context) throws IOException {
-      super.doSetNextReader(context);
-      afterDoc = after.doc - context.docBase;
-    }
-    
     @Override
     protected int topDocsSize() {
       return collectedHits < pq.size() ? collectedHits : pq.size();
     }
-    
+
     @Override
     protected TopDocs newTopDocs(ScoreDoc[] results, int start) {
       return results == null ? new TopDocs(totalHits, new ScoreDoc[0], Float.NaN) : new TopDocs(totalHits, results);
     }
+
+    @Override
+    public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+      final int docBase = context.docBase;
+      final int afterDoc = after.doc - context.docBase;
+      if (scoreDocsInOrder) {
+        return new ScorerLeafCollector(scoreDocsInOrder) {
+          @Override
+          public void collect(int doc) throws IOException {
+            float score = scorer.score();
+
+            // This collector cannot handle these scores:
+            assert score != Float.NEGATIVE_INFINITY;
+            assert !Float.isNaN(score);
+
+            totalHits++;
+
+            if (score > after.score || (score == after.score && doc <= afterDoc)) {
+              // hit was collected on a previous page
+              return;
+            }
+
+            if (score <= pqTop.score) {
+              // Since docs are returned in-order (i.e., increasing doc Id), a document
+              // with equal score to pqTop.score cannot compete since HitQueue favors
+              // documents with lower doc Ids. Therefore reject those docs too.
+              return;
+            }
+            collectedHits++;
+            pqTop.doc = doc + docBase;
+            pqTop.score = score;
+            pqTop = pq.updateTop();
+          }
+        };
+      } else {
+        return new ScorerLeafCollector(scoreDocsInOrder) {
+          @Override
+          public void collect(int doc) throws IOException {
+            float score = scorer.score();
+
+            // This collector cannot handle NaN
+            assert !Float.isNaN(score);
+
+            totalHits++;
+            if (score > after.score || (score == after.score && doc <= afterDoc)) {
+              // hit was collected on a previous page
+              return;
+            }
+            if (score < pqTop.score) {
+              // Doesn't compete w/ bottom entry in queue
+              return;
+            }
+            doc += docBase;
+            if (score == pqTop.score && doc > pqTop.doc) {
+              // Break tie in score by doc ID:
+              return;
+            }
+            collectedHits++;
+            pqTop.doc = doc;
+            pqTop.score = score;
+            pqTop = pq.updateTop();
+          }
+        };
+      }
+    }
   }
 
   /**
    * Creates a new {@link TopScoreDocCollector} given the number of hits to
    * collect and whether documents are scored in order by the input
-   * {@link Scorer} to {@link #setScorer(Scorer)}.
+   * {@link Scorer} to {@link LeafCollector#setScorer(Scorer)}.
    *
    * <p><b>NOTE</b>: The instances returned by this method
    * pre-allocate a full array of length
@@ -237,11 +228,11 @@ public abstract class TopScoreDocCollect
   public static TopScoreDocCollector create(int numHits, boolean docsScoredInOrder) {
     return create(numHits, null, docsScoredInOrder);
   }
-  
+
   /**
    * Creates a new {@link TopScoreDocCollector} given the number of hits to
    * collect, the bottom of the previous page, and whether documents are scored in order by the input
-   * {@link Scorer} to {@link #setScorer(Scorer)}.
+   * {@link Scorer} to {@link LeafCollector#setScorer(Scorer)}.
    *
    * <p><b>NOTE</b>: The instances returned by this method
    * pre-allocate a full array of length
@@ -249,27 +240,20 @@ public abstract class TopScoreDocCollect
    * objects.
    */
   public static TopScoreDocCollector create(int numHits, ScoreDoc after, boolean docsScoredInOrder) {
-    
+
     if (numHits <= 0) {
       throw new IllegalArgumentException("numHits must be > 0; please use TotalHitCountCollector if you just need the total hit count");
     }
-    
-    if (docsScoredInOrder) {
-      return after == null 
-        ? new InOrderTopScoreDocCollector(numHits) 
-        : new InOrderPagingScoreDocCollector(after, numHits);
+
+    if (after == null) {
+      return new SimpleTopScoreDocCollector(numHits, docsScoredInOrder);
     } else {
-      return after == null
-        ? new OutOfOrderTopScoreDocCollector(numHits)
-        : new OutOfOrderPagingScoreDocCollector(after, numHits);
+      return new PagingTopScoreDocCollector(numHits, docsScoredInOrder, after);
     }
-    
   }
-  
+
   ScoreDoc pqTop;
-  int docBase = 0;
-  Scorer scorer;
-    
+
   // prevents instantiation
   private TopScoreDocCollector(int numHits) {
     super(new HitQueue(numHits, true));
@@ -283,7 +267,7 @@ public abstract class TopScoreDocCollect
     if (results == null) {
       return EMPTY_TOPDOCS;
     }
-    
+
     // We need to compute maxScore in order to set it in TopDocs. If start == 0,
     // it means the largest element is already in results, use its score as
     // maxScore. Otherwise pop everything else, until the largest element is
@@ -295,17 +279,7 @@ public abstract class TopScoreDocCollect
       for (int i = pq.size(); i > 1; i--) { pq.pop(); }
       maxScore = pq.pop().score;
     }
-    
+
     return new TopDocs(totalHits, results, maxScore);
   }
-  
-  @Override
-  protected void doSetNextReader(LeafReaderContext context) throws IOException {
-    docBase = context.docBase;
-  }
-  
-  @Override
-  public void setScorer(Scorer scorer) throws IOException {
-    this.scorer = scorer;
-  }
 }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/JustCompileSearch.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/JustCompileSearch.java?rev=1649818&r1=1649817&r2=1649818&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/JustCompileSearch.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/JustCompileSearch.java Tue Jan  6 15:03:19 2015
@@ -99,42 +99,22 @@ final class JustCompileSearch {
   static final class JustCompileFieldComparator extends FieldComparator<Object> {
 
     @Override
-    public int compare(int slot1, int slot2) {
-      throw new UnsupportedOperationException(UNSUPPORTED_MSG);
-    }
-
-    @Override
-    public int compareBottom(int doc) {
-      throw new UnsupportedOperationException(UNSUPPORTED_MSG);
-    }
-
-    @Override
-    public void copy(int slot, int doc) {
-      throw new UnsupportedOperationException(UNSUPPORTED_MSG);
-    }
-
-    @Override
-    public void setBottom(int slot) {
-      throw new UnsupportedOperationException(UNSUPPORTED_MSG);
-    }
-
-    @Override
     public void setTopValue(Object value) {
       throw new UnsupportedOperationException(UNSUPPORTED_MSG);
     }
 
     @Override
-    public FieldComparator<Object> setNextReader(LeafReaderContext context) {
+    public Object value(int slot) {
       throw new UnsupportedOperationException(UNSUPPORTED_MSG);
     }
 
     @Override
-    public Object value(int slot) {
+    public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
       throw new UnsupportedOperationException(UNSUPPORTED_MSG);
     }
 
     @Override
-    public int compareTop(int doc) {
+    public int compare(int slot1, int slot2) {
       throw new UnsupportedOperationException(UNSUPPORTED_MSG);
     }
   }
@@ -260,23 +240,8 @@ final class JustCompileSearch {
     }
 
     @Override
-    public void collect(int doc) {
-      throw new UnsupportedOperationException(UNSUPPORTED_MSG);
-    }
-
-    @Override
-    protected void doSetNextReader(LeafReaderContext context) throws IOException {
-      throw new UnsupportedOperationException(UNSUPPORTED_MSG);
-    }
-
-    @Override
-    public void setScorer(Scorer scorer) {
-      throw new UnsupportedOperationException(UNSUPPORTED_MSG);
-    }
-    
-    @Override
-    public boolean acceptsDocsOutOfOrder() {
-      throw new UnsupportedOperationException(UNSUPPORTED_MSG);
+    public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+      throw new UnsupportedOperationException( UNSUPPORTED_MSG );
     }
 
     @Override

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestElevationComparator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestElevationComparator.java?rev=1649818&r1=1649817&r2=1649818&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestElevationComparator.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestElevationComparator.java Tue Jan  6 15:03:19 2015
@@ -27,6 +27,7 @@ import org.apache.lucene.search.similari
 import org.apache.lucene.store.*;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.BytesRef;
+
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
@@ -146,18 +147,54 @@ class ElevationComparatorSource extends
   public FieldComparator<Integer> newComparator(final String fieldname, final int numHits, int sortPos, boolean reversed) throws IOException {
    return new FieldComparator<Integer>() {
 
-     SortedDocValues idIndex;
      private final int[] values = new int[numHits];
      int bottomVal;
 
      @Override
-     public int compare(int slot1, int slot2) {
-       return values[slot2] - values[slot1];  // values will be small enough that there is no overflow concern
-     }
+    public LeafFieldComparator getLeafComparator(LeafReaderContext context)
+        throws IOException {
+      final SortedDocValues idIndex = DocValues.getSorted(context.reader(), fieldname);
+      return new LeafFieldComparator() {
+
+        @Override
+        public void setBottom(int slot) {
+          bottomVal = values[slot];
+        }
+
+        @Override
+        public int compareTop(int doc) {
+          throw new UnsupportedOperationException();
+        }
+
+        private int docVal(int doc) {
+          int ord = idIndex.getOrd(doc);
+          if (ord == -1) {
+            return 0;
+          } else {
+            final BytesRef term = idIndex.lookupOrd(ord);
+            Integer prio = priority.get(term);
+            return prio == null ? 0 : prio.intValue();
+          }
+        }
+
+        @Override
+        public int compareBottom(int doc) {
+          return docVal(doc) - bottomVal;
+        }
+
+        @Override
+        public void copy(int slot, int doc) {
+          values[slot] = docVal(doc);
+        }
+
+        @Override
+        public void setScorer(Scorer scorer) {}
+      };
+    }
 
      @Override
-     public void setBottom(int slot) {
-       bottomVal = values[slot];
+     public int compare(int slot1, int slot2) {
+       return values[slot2] - values[slot1];  // values will be small enough that there is no overflow concern
      }
 
      @Override
@@ -165,42 +202,12 @@ class ElevationComparatorSource extends
        throw new UnsupportedOperationException();
      }
 
-     private int docVal(int doc) {
-       int ord = idIndex.getOrd(doc);
-       if (ord == -1) {
-         return 0;
-       } else {
-         final BytesRef term = idIndex.lookupOrd(ord);
-         Integer prio = priority.get(term);
-         return prio == null ? 0 : prio.intValue();
-       }
-     }
-
-     @Override
-     public int compareBottom(int doc) {
-       return docVal(doc) - bottomVal;
-     }
-
-     @Override
-     public void copy(int slot, int doc) {
-       values[slot] = docVal(doc);
-     }
-
-     @Override
-     public FieldComparator<Integer> setNextReader(LeafReaderContext context) throws IOException {
-       idIndex = DocValues.getSorted(context.reader(), fieldname);
-       return this;
-     }
-
      @Override
      public Integer value(int slot) {
        return Integer.valueOf(values[slot]);
      }
 
-     @Override
-     public int compareTop(int doc) {
-       throw new UnsupportedOperationException();
-     }
+
    };
  }
 }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestSort.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestSort.java?rev=1649818&r1=1649817&r2=1649818&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestSort.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestSort.java Tue Jan  6 15:03:19 2015
@@ -785,4 +785,63 @@ public class TestSort extends LuceneTest
     ir.close();
     dir.close();
   }
+
+  /** Tests sorting on multiple sort fields */
+  public void testMultiSort() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+    Document doc = new Document();
+    doc.add(new SortedDocValuesField("value1", new BytesRef("foo")));
+    doc.add(new NumericDocValuesField("value2", 0));
+    doc.add(newStringField("value1", "foo", Field.Store.YES));
+    doc.add(newStringField("value2", "0", Field.Store.YES));
+    writer.addDocument(doc);
+    doc = new Document();
+    doc.add(new SortedDocValuesField("value1", new BytesRef("bar")));
+    doc.add(new NumericDocValuesField("value2", 1));
+    doc.add(newStringField("value1", "bar", Field.Store.YES));
+    doc.add(newStringField("value2", "1", Field.Store.YES));
+    writer.addDocument(doc);
+    doc = new Document();
+    doc.add(new SortedDocValuesField("value1", new BytesRef("bar")));
+    doc.add(new NumericDocValuesField("value2", 0));
+    doc.add(newStringField("value1", "bar", Field.Store.YES));
+    doc.add(newStringField("value2", "0", Field.Store.YES));
+    writer.addDocument(doc);
+    doc = new Document();
+    doc.add(new SortedDocValuesField("value1", new BytesRef("foo")));
+    doc.add(new NumericDocValuesField("value2", 1));
+    doc.add(newStringField("value1", "foo", Field.Store.YES));
+    doc.add(newStringField("value2", "1", Field.Store.YES));
+    writer.addDocument(doc);
+    IndexReader ir = writer.getReader();
+    writer.close();
+    
+    IndexSearcher searcher = newSearcher(ir);
+    Sort sort = new Sort(
+        new SortField("value1", SortField.Type.STRING),
+        new SortField("value2", SortField.Type.LONG));
+
+    TopDocs td = searcher.search(new MatchAllDocsQuery(), 10, sort);
+    assertEquals(4, td.totalHits);
+    // 'bar' comes before 'foo'
+    assertEquals("bar", searcher.doc(td.scoreDocs[0].doc).get("value1"));
+    assertEquals("bar", searcher.doc(td.scoreDocs[1].doc).get("value1"));
+    assertEquals("foo", searcher.doc(td.scoreDocs[2].doc).get("value1"));
+    assertEquals("foo", searcher.doc(td.scoreDocs[3].doc).get("value1"));
+    // 0 comes before 1
+    assertEquals("0", searcher.doc(td.scoreDocs[0].doc).get("value2"));
+    assertEquals("1", searcher.doc(td.scoreDocs[1].doc).get("value2"));
+    assertEquals("0", searcher.doc(td.scoreDocs[2].doc).get("value2"));
+    assertEquals("1", searcher.doc(td.scoreDocs[3].doc).get("value2"));
+
+    // Now with overflow
+    td = searcher.search(new MatchAllDocsQuery(), 1, sort);
+    assertEquals(4, td.totalHits);
+    assertEquals("bar", searcher.doc(td.scoreDocs[0].doc).get("value1"));
+    assertEquals("0", searcher.doc(td.scoreDocs[0].doc).get("value2"));
+
+    ir.close();
+    dir.close();
+  }
 }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java?rev=1649818&r1=1649817&r2=1649818&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java Tue Jan  6 15:03:19 2015
@@ -31,7 +31,6 @@ public class TestTopDocsCollector extend
   private static final class MyTopsDocCollector extends TopDocsCollector<ScoreDoc> {
 
     private int idx = 0;
-    private int base = 0;
     
     public MyTopsDocCollector(int size) {
       super(new HitQueue(size, false));
@@ -55,24 +54,26 @@ public class TestTopDocsCollector extend
     }
     
     @Override
-    public void collect(int doc) {
-      ++totalHits;
-      pq.insertWithOverflow(new ScoreDoc(doc + base, scores[idx++]));
-    }
-
-    @Override
-    protected void doSetNextReader(LeafReaderContext context) throws IOException {
-      base = context.docBase;
-    }
+    public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+      final int base = context.docBase;
+      return new LeafCollector() {
+        
+        @Override
+        public void collect(int doc) {
+          ++totalHits;
+          pq.insertWithOverflow(new ScoreDoc(doc + base, scores[idx++]));
+        }
 
-    @Override
-    public void setScorer(Scorer scorer) {
-      // Don't do anything. Assign scores in random
-    }
-    
-    @Override
-    public boolean acceptsDocsOutOfOrder() {
-      return true;
+        @Override
+        public void setScorer(Scorer scorer) {
+          // Don't do anything. Assign scores in random
+        }
+        
+        @Override
+        public boolean acceptsDocsOutOfOrder() {
+          return true;
+        }
+      };
     }
 
   }