You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2015/02/09 00:53:25 UTC

svn commit: r1658277 [7/38] - in /lucene/dev/branches/lucene6005: ./ dev-tools/ dev-tools/idea/solr/contrib/dataimporthandler/ dev-tools/idea/solr/contrib/velocity/ dev-tools/maven/lucene/replicator/ dev-tools/maven/solr/ dev-tools/maven/solr/contrib/v...

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java Sun Feb  8 23:53:14 2015
@@ -27,816 +27,378 @@ import org.apache.lucene.util.PriorityQu
  * A {@link Collector} that sorts by {@link SortField} using
  * {@link FieldComparator}s.
  * <p/>
- * See the {@link #create(org.apache.lucene.search.Sort, int, boolean, boolean, boolean, boolean)} method
+ * See the {@link #create(org.apache.lucene.search.Sort, int, 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);
-        }
-      }
-    }
-    
-    @Override
-    protected void doSetNextReader(LeafReaderContext context) throws IOException {
-      this.docBase = context.docBase;
-      queue.setComparator(0, comparator.setNextReader(context));
-      comparator = queue.firstComparator;
+    OneComparatorLeafCollector(LeafFieldComparator comparator, int reverseMul) {
+      this.comparator = comparator;
+      this.reverseMul = reverseMul;
     }
-    
+
     @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;
-    }
-
-  }
-
-  /*
-   * 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;
+      for (LeafFieldComparator comparator : comparators) {
+        comparator.setScorer(scorer);
       }
-      ++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);
-        }
-      }
-    }
-    
-    @Override
-    public boolean acceptsDocsOutOfOrder() {
-      return true;
     }
-
   }
 
   /*
-   * 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);
-        }
-      } 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 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);
+              }
+            }
           }
-        }
-      }
-    }
 
-    @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));
-      }
-    }
+        };
+      } else {
+        return new MultiComparatorLeafCollector(comparators, reverseMul) {
 
-    @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
-   * 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) {
-      super(queue, numHits, fillFields);
-    }
-    
-    @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;
+          @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);
+              }
             }
-            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);
-
-        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;
-    }
 
   }
 
   /*
-   * 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);
-        }
-      } 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 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);
+              }
+            }
           }
-        }
-      }
-    }
 
-    @Override
-    public void setScorer(Scorer scorer) throws IOException {
-      this.scorer = scorer;
-      super.setScorer(scorer);
-    }
-  }
+        };
+      } else {
+        return new MultiComparatorLeafCollector(comparators, reverseMul) {
 
-  /*
-   * Implements a TopFieldCollector over multiple SortField criteria, with
-   * tracking document scores and 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) {
-      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
-        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;
+          @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);
+              }
             }
-            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);
 
-        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;
-    }
 
   }
 
   /*
-   * 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;
-          }
-        }
-
-        // This hit is competitive - replace bottom element in queue & adjustTop
-        for (int i = 0; i < comparators.length; i++) {
-          comparators[i].copy(bottom.slot, doc);
-        }
-
-        // Compute score only if it is competitive.
-        final float score = scorer.score();
-        updateBottom(doc, score);
+    public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+      docBase = context.docBase;
 
-        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);
-        }
+      final LeafFieldComparator[] comparators = queue.getComparators(context);
+      final int[] reverseMul = queue.getReverseMul();
 
-        // 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);
-          }
-        }
-      }
-    }
+      if (comparators.length == 1) {
+        return new OneComparatorLeafCollector(comparators[0], reverseMul[0]) {
 
-    @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.
-   */
-  private final static class OutOfOrderMultiComparatorScoringNoMaxScoreCollector
-      extends MultiComparatorScoringNoMaxScoreCollector {
-    
-    public OutOfOrderMultiComparatorScoringNoMaxScoreCollector(
-        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
-        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;
+          @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);
+              }
             }
-            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);
-        }
-
-        // Compute score only if it is competitive.
-        final float score = scorer.score();
-        updateBottom(doc, score);
-
-        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);
-        }
+        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);
-    }
-    
-    @Override
-    public boolean acceptsDocsOutOfOrder() {
-      return true;
+        };
+      }
     }
 
   }
@@ -846,30 +408,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 +433,81 @@ 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) {
+              // not competitive since documents are visited in doc id order
+              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
-    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 +520,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 +539,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
@@ -1065,18 +561,14 @@ public abstract class TopFieldCollector
    *          true affects performance as it incurs the score computation on
    *          each result. Also, setting this true automatically sets
    *          <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)}.
    * @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
    */
   public static TopFieldCollector create(Sort sort, int numHits,
-      boolean fillFields, boolean trackDocScores, boolean trackMaxScore,
-      boolean docsScoredInOrder)
+      boolean fillFields, boolean trackDocScores, boolean trackMaxScore)
       throws IOException {
-    return create(sort, numHits, null, fillFields, trackDocScores, trackMaxScore, docsScoredInOrder);
+    return create(sort, numHits, null, fillFields, trackDocScores, trackMaxScore);
   }
 
   /**
@@ -1086,7 +578,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
@@ -1110,22 +602,18 @@ public abstract class TopFieldCollector
    *          true affects performance as it incurs the score computation on
    *          each result. Also, setting this true automatically sets
    *          <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)}.
    * @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
    */
   public static TopFieldCollector create(Sort sort, int numHits, FieldDoc after,
-      boolean fillFields, boolean trackDocScores, boolean trackMaxScore,
-      boolean docsScoredInOrder)
+      boolean fillFields, boolean trackDocScores, boolean trackMaxScore)
       throws IOException {
 
     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,43 +621,12 @@ 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);
-        } else if (trackDocScores) {
-          return new MultiComparatorScoringNoMaxScoreCollector(queue, numHits, fillFields);
-        } else {
-          return new MultiComparatorNonScoringCollector(queue, numHits, fillFields);
-        }
+      if (trackMaxScore) {
+        return new ScoringMaxScoreCollector(queue, numHits, fillFields);
+      } else if (trackDocScores) {
+        return new ScoringNoMaxScoreCollector(queue, numHits, fillFields);
       } else {
-        if (trackMaxScore) {
-          return new OutOfOrderMultiComparatorScoringMaxScoreCollector(queue, numHits, fillFields);
-        } else if (trackDocScores) {
-          return new OutOfOrderMultiComparatorScoringNoMaxScoreCollector(queue, numHits, fillFields);
-        } else {
-          return new OutOfOrderMultiComparatorNonScoringCollector(queue, numHits, fillFields);
-        }
+        return new NonScoringCollector(queue, numHits, fillFields);
       }
     } else {
       if (after.fields == null) {
@@ -1183,12 +640,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 +678,7 @@ public abstract class TopFieldCollector
       }
     }
   }
-  
+
   @Override
   protected TopDocs newTopDocs(ScoreDoc[] results, int start) {
     if (results == null) {
@@ -1218,12 +687,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/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java Sun Feb  8 23:53:14 2015
@@ -36,242 +36,151 @@ 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();
+  abstract static class ScorerLeafCollector implements LeafCollector {
+
+    Scorer scorer;
 
-      // 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;
+    public void setScorer(Scorer scorer) throws IOException {
+      this.scorer = scorer;
     }
+
   }
-  
-  // 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;
 
-    private InOrderPagingScoreDocCollector(ScoreDoc after, int numHits) {
+  private static class SimpleTopScoreDocCollector extends TopScoreDocCollector {
+
+    SimpleTopScoreDocCollector(int numHits) {
       super(numHits);
-      this.after = after;
     }
-    
+
     @Override
-    public void collect(int doc) throws IOException {
-      float score = scorer.score();
+    public LeafCollector getLeafCollector(LeafReaderContext context)
+        throws IOException {
+      final int docBase = context.docBase;
+      return new ScorerLeafCollector() {
 
-      // 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();
-    }
+        @Override
+        public void collect(int doc) throws IOException {
+          float score = scorer.score();
 
-    @Override
-    public boolean acceptsDocsOutOfOrder() {
-      return false;
-    }
+          // This collector cannot handle these scores:
+          assert score != Float.NEGATIVE_INFINITY;
+          assert !Float.isNaN(score);
 
-    @Override
-    protected void doSetNextReader(LeafReaderContext context) throws IOException {
-      super.doSetNextReader(context);
-      afterDoc = after.doc - context.docBase;
-    }
+          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
-    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) {
-      super(numHits);
-    }
-    
-    @Override
-    public void collect(int doc) throws IOException {
-      float score = scorer.score();
+  }
 
-      // This collector cannot handle NaN
-      assert !Float.isNaN(score);
+  private static class PagingTopScoreDocCollector extends TopScoreDocCollector {
 
-      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();
-    }
-    
-    @Override
-    public boolean acceptsDocsOutOfOrder() {
-      return true;
-    }
-  }
-  
-  // Assumes docs are scored out of order.
-  private static class OutOfOrderPagingScoreDocCollector 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;
 
-    private OutOfOrderPagingScoreDocCollector(ScoreDoc after, int numHits) {
+    PagingTopScoreDocCollector(int numHits, ScoreDoc after) {
       super(numHits);
       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;
+      return new ScorerLeafCollector() {
+        @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();
+        }
+      };
+    }
   }
 
   /**
    * 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
    * <code>numHits</code>, and fill the array with sentinel
    * objects.
    */
-  public static TopScoreDocCollector create(int numHits, boolean docsScoredInOrder) {
-    return create(numHits, null, docsScoredInOrder);
+  public static TopScoreDocCollector create(int numHits) {
+    return create(numHits, null);
   }
-  
+
   /**
    * 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
    * <code>numHits</code>, and fill the array with sentinel
    * objects.
    */
-  public static TopScoreDocCollector create(int numHits, ScoreDoc after, boolean docsScoredInOrder) {
-    
+  public static TopScoreDocCollector create(int numHits, ScoreDoc after) {
+
     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);
     } else {
-      return after == null
-        ? new OutOfOrderTopScoreDocCollector(numHits)
-        : new OutOfOrderPagingScoreDocCollector(after, numHits);
+      return new PagingTopScoreDocCollector(numHits, after);
     }
-    
   }
-  
+
   ScoreDoc pqTop;
-  int docBase = 0;
-  Scorer scorer;
-    
+
   // prevents instantiation
-  private TopScoreDocCollector(int numHits) {
+  TopScoreDocCollector(int numHits) {
     super(new HitQueue(numHits, true));
     // HitQueue implements getSentinelObject to return a ScoreDoc, so we know
     // that at this point top() is already initialized.
@@ -283,7 +192,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 +204,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/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java Sun Feb  8 23:53:14 2015
@@ -34,9 +34,4 @@ public class TotalHitCountCollector exte
   public void collect(int doc) {
     totalHits++;
   }
-
-  @Override
-  public boolean acceptsDocsOutOfOrder() {
-    return true;
-  }
 }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/Weight.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/Weight.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/Weight.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/Weight.java Sun Feb  8 23:53:14 2015
@@ -79,11 +79,6 @@ public abstract class Weight {
    * Returns a {@link Scorer} which scores documents in/out-of order according
    * to <code>scoreDocsInOrder</code>.
    * <p>
-   * <b>NOTE:</b> even if <code>scoreDocsInOrder</code> is false, it is
-   * recommended to check whether the returned <code>Scorer</code> indeed scores
-   * documents out of order (i.e., call {@link #scoresDocsOutOfOrder()}), as
-   * some <code>Scorer</code> implementations will always return documents
-   * in-order.<br>
    * <b>NOTE:</b> null can be returned if no documents will be scored by this
    * query.
    * 
@@ -108,15 +103,6 @@ public abstract class Weight {
    *
    * @param context
    *          the {@link org.apache.lucene.index.LeafReaderContext} for which to return the {@link Scorer}.
-   * @param scoreDocsInOrder
-   *          specifies whether in-order scoring of documents is required. Note
-   *          that if set to false (i.e., out-of-order scoring is required),
-   *          this method can return whatever scoring mode it supports, as every
-   *          in-order scorer is also an out-of-order one. However, an
-   *          out-of-order scorer may not support {@link Scorer#nextDoc()}
-   *          and/or {@link Scorer#advance(int)}, therefore it is recommended to
-   *          request an in-order scorer if use of these
-   *          methods is required.
    * @param acceptDocs
    *          Bits that represent the allowable docs to match (typically deleted docs
    *          but possibly filtering other documents)
@@ -125,7 +111,7 @@ public abstract class Weight {
    * passes them to a collector.
    * @throws IOException if there is a low-level I/O error
    */
-  public BulkScorer bulkScorer(LeafReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
+  public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
 
     Scorer scorer = scorer(context, acceptDocs);
     if (scorer == null) {
@@ -150,7 +136,7 @@ public abstract class Weight {
     }
 
     @Override
-    public boolean score(LeafCollector collector, int max) throws IOException {
+    public int score(LeafCollector collector, int min, int max) throws IOException {
       // TODO: this may be sort of weird, when we are
       // embedded in a BooleanScorer, because we are
       // called for every chunk of 2048 documents.  But,
@@ -158,13 +144,13 @@ public abstract class Weight {
       // Collector doing something "interesting" in
       // setScorer will be forced to use BS2 anyways:
       collector.setScorer(scorer);
-      if (max == DocIdSetIterator.NO_MORE_DOCS) {
+      if (scorer.docID() == -1 && min == 0 && max == DocIdSetIterator.NO_MORE_DOCS) {
         scoreAll(collector, scorer);
-        return false;
+        return DocIdSetIterator.NO_MORE_DOCS;
       } else {
         int doc = scorer.docID();
-        if (doc < 0) {
-          doc = scorer.nextDoc();
+        if (doc < min) {
+          doc = scorer.advance(min);
         }
         return scoreRange(collector, scorer, doc, max);
       }
@@ -174,12 +160,12 @@ public abstract class Weight {
      *  separate this from {@link #scoreAll} to help out
      *  hotspot.
      *  See <a href="https://issues.apache.org/jira/browse/LUCENE-5487">LUCENE-5487</a> */
-    static boolean scoreRange(LeafCollector collector, Scorer scorer, int currentDoc, int end) throws IOException {
+    static int scoreRange(LeafCollector collector, Scorer scorer, int currentDoc, int end) throws IOException {
       while (currentDoc < end) {
         collector.collect(currentDoc);
         currentDoc = scorer.nextDoc();
       }
-      return currentDoc != DocIdSetIterator.NO_MORE_DOCS;
+      return currentDoc;
     }
     
     /** Specialized method to bulk-score all hits; we
@@ -187,25 +173,9 @@ public abstract class Weight {
      *  hotspot.
      *  See <a href="https://issues.apache.org/jira/browse/LUCENE-5487">LUCENE-5487</a> */
     static void scoreAll(LeafCollector collector, Scorer scorer) throws IOException {
-      int doc;
-      while ((doc = scorer.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+      for (int doc = scorer.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = scorer.nextDoc()) {
         collector.collect(doc);
       }
     }
   }
-
-  /**
-   * Returns true iff this implementation scores docs only out of order. This
-   * method is used in conjunction with {@link Collector}'s
-   * {@link LeafCollector#acceptsDocsOutOfOrder() acceptsDocsOutOfOrder} and
-   * {@link #bulkScorer(org.apache.lucene.index.LeafReaderContext, boolean, Bits)} to
-   * create a matching {@link Scorer} instance for a given {@link Collector}, or
-   * vice versa.
-   * <p>
-   * <b>NOTE:</b> the default implementation returns <code>false</code>, i.e.
-   * the <code>Scorer</code> scores documents in-order.
-   */
-  public boolean scoresDocsOutOfOrder() {
-    return false;
-  }
 }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/package.html?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/package.html (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/package.html Sun Feb  8 23:53:14 2015
@@ -443,8 +443,8 @@ on the built-in available scoring models
                 given the Query.
             </li>
             <li>
-                {@link org.apache.lucene.search.Weight#bulkScorer(org.apache.lucene.index.LeafReaderContext, boolean, org.apache.lucene.util.Bits)
-                  scorer(LeafReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs)} &mdash;
+                {@link org.apache.lucene.search.Weight#bulkScorer(org.apache.lucene.index.LeafReaderContext, org.apache.lucene.util.Bits)
+                  scorer(LeafReaderContext context, Bits acceptDocs)} &mdash;
                 Construct a new {@link org.apache.lucene.search.BulkScorer BulkScorer} for this Weight. See <a href="#bulkScorerClass">The BulkScorer Class</a>
                 below for help defining a BulkScorer. This is an optional method, and most queries do not implement it.
             </li>
@@ -508,7 +508,7 @@ on the built-in available scoring models
         abstract method:
         <ol>
             <li>
-                {@link org.apache.lucene.search.BulkScorer#score(org.apache.lucene.search.LeafCollector,int) score(LeafCollector,int)} &mdash;
+                {@link org.apache.lucene.search.BulkScorer#score(org.apache.lucene.search.LeafCollector,int,int) score(LeafCollector,int,int)} &mdash;
 		Score all documents up to but not including the specified max document.
 	    </li>
         </ol>

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/ByteBufferIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/ByteBufferIndexInput.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/ByteBufferIndexInput.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/ByteBufferIndexInput.java Sun Feb  8 23:53:14 2015
@@ -170,9 +170,7 @@ abstract class ByteBufferIndexInput exte
         this.curBufIndex = bi;
         this.curBuf = b;
       }
-    } catch (ArrayIndexOutOfBoundsException aioobe) {
-      throw new EOFException("seek past EOF: " + this);
-    } catch (IllegalArgumentException iae) {
+    } catch (ArrayIndexOutOfBoundsException | IllegalArgumentException e) {
       throw new EOFException("seek past EOF: " + this);
     } catch (NullPointerException npe) {
       throw new AlreadyClosedException("Already closed: " + this);
@@ -198,9 +196,7 @@ abstract class ByteBufferIndexInput exte
       b.position((int) (pos & chunkSizeMask));
       this.curBufIndex = bi;
       this.curBuf = b;
-    } catch (ArrayIndexOutOfBoundsException aioobe) {
-      throw new EOFException("seek past EOF: " + this);
-    } catch (IllegalArgumentException iae) {
+    } catch (ArrayIndexOutOfBoundsException | IllegalArgumentException aioobe) {
       throw new EOFException("seek past EOF: " + this);
     } catch (NullPointerException npe) {
       throw new AlreadyClosedException("Already closed: " + this);

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/FilterDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/FilterDirectory.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/FilterDirectory.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/FilterDirectory.java Sun Feb  8 23:53:14 2015
@@ -23,7 +23,7 @@ import java.util.Collection;
 /** Directory implementation that delegates calls to another directory.
  *  This class can be used to add limitations on top of an existing
  *  {@link Directory} implementation such as
- *  {@link RateLimitedDirectoryWrapper rate limiting} or to add additional
+ *  {@link NRTCachingDirectory} or to add additional
  *  sanity checks for tests. However, if you plan to write your own
  *  {@link Directory} implementation, you should consider extending directly
  *  {@link Directory} or {@link BaseDirectory} rather than try to reuse

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/LockStressTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/LockStressTest.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/LockStressTest.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/LockStressTest.java Sun Feb  8 23:53:14 2015
@@ -139,7 +139,7 @@ public class LockStressTest {
     // try to create a new instance
     try {
       return Class.forName(lockFactoryClassName).asSubclass(FSLockFactory.class).newInstance();
-    } catch (IllegalAccessException | InstantiationException | ClassCastException | ClassNotFoundException e) {
+    } catch (ReflectiveOperationException | ClassCastException e) {
       // fall-through
     }
 

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RateLimitedIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RateLimitedIndexOutput.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RateLimitedIndexOutput.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RateLimitedIndexOutput.java Sun Feb  8 23:53:14 2015
@@ -24,7 +24,8 @@ import java.io.IOException;
  * 
  * @lucene.internal
  */
-final class RateLimitedIndexOutput extends IndexOutput {
+
+public final class RateLimitedIndexOutput extends IndexOutput {
   
   private final IndexOutput delegate;
   private final RateLimiter rateLimiter;
@@ -36,7 +37,7 @@ final class RateLimitedIndexOutput exten
    * which does volatile read. */
   private long currentMinPauseCheckBytes;
 
-  RateLimitedIndexOutput(final RateLimiter rateLimiter, final IndexOutput delegate) {
+  public RateLimitedIndexOutput(final RateLimiter rateLimiter, final IndexOutput delegate) {
     super("RateLimitedIndexOutput(" + delegate + ")");
     this.delegate = delegate;
     this.rateLimiter = rateLimiter;
@@ -72,7 +73,7 @@ final class RateLimitedIndexOutput exten
     delegate.writeBytes(b, offset, length);
   }
   
-  private void checkRate() {
+  private void checkRate() throws IOException {
     if (bytesSinceLastPause > currentMinPauseCheckBytes) {
       rateLimiter.pause(bytesSinceLastPause);
       bytesSinceLastPause = 0;

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RateLimiter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RateLimiter.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RateLimiter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RateLimiter.java Sun Feb  8 23:53:14 2015
@@ -17,6 +17,8 @@ package org.apache.lucene.store;
  * limitations under the License.
  */
 
+import java.io.IOException;
+
 import org.apache.lucene.util.ThreadInterruptedException;
 
 /** Abstract base class to rate limit IO.  Typically implementations are
@@ -27,14 +29,14 @@ import org.apache.lucene.util.ThreadInte
 public abstract class RateLimiter {
 
   /**
-   * Sets an updated mb per second rate limit.
+   * Sets an updated MB per second rate limit.
    */
-  public abstract void setMbPerSec(double mbPerSec);
+  public abstract void setMBPerSec(double mbPerSec);
 
   /**
-   * The current mb per second rate limit.
+   * The current MB per second rate limit.
    */
-  public abstract double getMbPerSec();
+  public abstract double getMBPerSec();
   
   /** Pauses, if necessary, to keep the instantaneous IO
    *  rate at or below the target. 
@@ -43,7 +45,7 @@ public abstract class RateLimiter {
    *  </p>
    *  @return the pause time in nano seconds 
    * */
-  public abstract long pause(long bytes);
+  public abstract long pause(long bytes) throws IOException;
   
   /** How many bytes caller should add up itself before invoking {@link #pause}. */
   public abstract long getMinPauseCheckBytes();
@@ -65,7 +67,7 @@ public abstract class RateLimiter {
 
     /** mbPerSec is the MB/sec max IO rate */
     public SimpleRateLimiter(double mbPerSec) {
-      setMbPerSec(mbPerSec);
+      setMBPerSec(mbPerSec);
       lastNS = System.nanoTime();
     }
 
@@ -73,7 +75,7 @@ public abstract class RateLimiter {
      * Sets an updated mb per second rate limit.
      */
     @Override
-    public void setMbPerSec(double mbPerSec) {
+    public void setMBPerSec(double mbPerSec) {
       this.mbPerSec = mbPerSec;
       minPauseCheckBytes = (long) ((MIN_PAUSE_CHECK_MSEC / 1000.0) * mbPerSec * 1024 * 1024);
     }
@@ -87,7 +89,7 @@ public abstract class RateLimiter {
      * The current mb per second rate limit.
      */
     @Override
-    public double getMbPerSec() {
+    public double getMBPerSec() {
       return this.mbPerSec;
     }
     

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/IOUtils.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/IOUtils.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/IOUtils.java Sun Feb  8 23:53:14 2015
@@ -407,7 +407,8 @@ public final class IOUtils {
     }
     
     if (isDir) {
-      assert (Constants.LINUX || Constants.MAC_OS_X) == false :
+      // TODO: LUCENE-6169 - Fix this assert once Java 9 problems are solved!
+      assert (Constants.LINUX || Constants.MAC_OS_X) == false || Constants.JRE_IS_MINIMUM_JAVA9 :
         "On Linux and MacOSX fsyncing a directory should not throw IOException, "+
         "we just don't want to rely on that in production (undocumented). Got: " + exc;
       // Ignore exception if it is a directory

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/PrintStreamInfoStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/PrintStreamInfoStream.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/PrintStreamInfoStream.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/PrintStreamInfoStream.java Sun Feb  8 23:53:14 2015
@@ -19,7 +19,10 @@ package org.apache.lucene.util;
 
 import java.io.IOException;
 import java.io.PrintStream;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
 import java.util.Date;
+import java.util.Locale;
 import java.util.concurrent.atomic.AtomicInteger;
 
 /**
@@ -32,6 +35,8 @@ public class PrintStreamInfoStream exten
   // Used for printing messages
   private static final AtomicInteger MESSAGE_ID = new AtomicInteger();
   protected final int messageID;
+
+  private static final DateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS", Locale.ROOT);
   
   protected final PrintStream stream;
   
@@ -46,7 +51,7 @@ public class PrintStreamInfoStream exten
   
   @Override
   public void message(String component, String message) {
-    stream.println(component + " " + messageID + " [" + new Date() + "; " + Thread.currentThread().getName() + "]: " + message);    
+    stream.println(component + " " + messageID + " [" + dateFormat.format(new Date()) + "; " + Thread.currentThread().getName() + "]: " + message);    
   }
 
   @Override

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/StringHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/StringHelper.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/StringHelper.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/StringHelper.java Sun Feb  8 23:53:14 2015
@@ -253,7 +253,7 @@ public abstract class StringHelper {
       x0 = Long.parseLong(prop, 16);
       x1 = x0;
     } else {
-      // "Rough randomess" from 3 different sources:
+      // Randomess from 3 different sources:
       x0 = System.nanoTime();
       x1 = StringHelper.class.hashCode() << 32;
       StringBuilder sb = new StringBuilder();

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/Version.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/Version.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/Version.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/Version.java Sun Feb  8 23:53:14 2015
@@ -39,6 +39,13 @@ public final class Version {
   @Deprecated
   public static final Version LUCENE_5_0_0 = new Version(5, 0, 0);
 
+  /**
+   * Match settings and bugs in Lucene's 5.1.0 release.
+   * @deprecated Use latest
+   */
+  @Deprecated
+  public static final Version LUCENE_5_1_0 = new Version(5, 1, 0);
+
   /** Match settings and bugs in Lucene's 6.0 release.
    *  <p>
    *  Use this to get the latest &amp; greatest settings, bug

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/automaton/Automaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/automaton/Automaton.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/automaton/Automaton.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/automaton/Automaton.java Sun Feb  8 23:53:14 2015
@@ -19,16 +19,21 @@ package org.apache.lucene.util.automaton
 
 //import java.io.IOException;
 //import java.io.PrintWriter;
+
 import java.util.Arrays;
 import java.util.BitSet;
 import java.util.HashSet;
 import java.util.Set;
 
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.InPlaceMergeSorter;
+import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.Sorter;
 
 
+
+
 // TODO
 //   - could use packed int arrays instead
 //   - could encode dest w/ delta from to?
@@ -47,7 +52,8 @@ import org.apache.lucene.util.Sorter;
  *
  * @lucene.experimental */
 
-public class Automaton {
+public class Automaton implements Accountable {
+
   /** Where we next write to the int[] states; this increments by 2 for
    *  each added state because we pack a pointer to the transitions
    *  array and a count of how many transitions leave the state.  */
@@ -879,4 +885,14 @@ public class Automaton {
       }
     }
   }
+
+  @Override
+  public long ramBytesUsed() {
+    // TODO: BitSet RAM usage (isAccept.size()/8) isn't fully accurate...
+    return RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + RamUsageEstimator.sizeOf(states) + RamUsageEstimator.sizeOf(transitions) +
+      RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + (isAccept.size() / 8) + RamUsageEstimator.NUM_BYTES_OBJECT_REF +
+      2 * RamUsageEstimator.NUM_BYTES_OBJECT_REF +
+      3 * RamUsageEstimator.NUM_BYTES_INT +
+      RamUsageEstimator.NUM_BYTES_BOOLEAN;
+  }
 }