lucene-java-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From mikemcc...@apache.org
Subject svn commit: r773194 [1/2] - in /lucene/java/trunk: ./ contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ src/demo/org/apache/lucene/demo/ src/java/org/apache/lucene/search/ src/java/org/apache/lucene/util/ src/test/org/apache/lucene/i...
Date Sat, 09 May 2009 10:36:42 GMT
Author: mikemccand
Date: Sat May  9 10:36:41 2009
New Revision: 773194

URL: http://svn.apache.org/viewvc?rev=773194&view=rev
Log:
LUCENE-1593: optimize core Collectors for in-docID-order scorer case

Added:
    lucene/java/trunk/src/test/org/apache/lucene/search/TestTopScoreDocCollector.java
Modified:
    lucene/java/trunk/CHANGES.txt
    lucene/java/trunk/common-build.xml
    lucene/java/trunk/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ReadTask.java
    lucene/java/trunk/src/demo/org/apache/lucene/demo/SearchFiles.java
    lucene/java/trunk/src/java/org/apache/lucene/search/BooleanQuery.java
    lucene/java/trunk/src/java/org/apache/lucene/search/BooleanScorer.java
    lucene/java/trunk/src/java/org/apache/lucene/search/BooleanScorer2.java
    lucene/java/trunk/src/java/org/apache/lucene/search/FieldDoc.java
    lucene/java/trunk/src/java/org/apache/lucene/search/HitQueue.java
    lucene/java/trunk/src/java/org/apache/lucene/search/IndexSearcher.java
    lucene/java/trunk/src/java/org/apache/lucene/search/MultiSearcher.java
    lucene/java/trunk/src/java/org/apache/lucene/search/ParallelMultiSearcher.java
    lucene/java/trunk/src/java/org/apache/lucene/search/ScoreDoc.java
    lucene/java/trunk/src/java/org/apache/lucene/search/Sort.java
    lucene/java/trunk/src/java/org/apache/lucene/search/TopDocCollector.java
    lucene/java/trunk/src/java/org/apache/lucene/search/TopDocsCollector.java
    lucene/java/trunk/src/java/org/apache/lucene/search/TopFieldCollector.java
    lucene/java/trunk/src/java/org/apache/lucene/search/TopScoreDocCollector.java
    lucene/java/trunk/src/java/org/apache/lucene/util/PriorityQueue.java
    lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReader.java
    lucene/java/trunk/src/test/org/apache/lucene/search/TestPositiveScoresOnlyCollector.java
    lucene/java/trunk/src/test/org/apache/lucene/search/TestSort.java
    lucene/java/trunk/src/test/org/apache/lucene/search/TestTopDocsCollector.java

Modified: lucene/java/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/java/trunk/CHANGES.txt?rev=773194&r1=773193&r2=773194&view=diff
==============================================================================
--- lucene/java/trunk/CHANGES.txt (original)
+++ lucene/java/trunk/CHANGES.txt Sat May  9 10:36:41 2009
@@ -62,6 +62,11 @@
     writes an empty commit when it's created.  (Paul Taylor via Mike
     McCandless)
 
+ 5. LUCENE-1593: When you call Sort() or Sort.setSort(String field,
+    boolean reverse), the resulting SortField array no longer ends
+    with SortField.FIELD_DOC (it was unnecessary as Lucene breaks ties
+    internally by docID). (Shai Erera via Michael McCandless)
+
 API Changes
 
 1. LUCENE-1419: Add expert API to set custom indexing chain. This API is 
@@ -167,6 +172,10 @@
    contain field names with non-ascii characters.  (Mike Streeton via
    Mike McCandless)
 
+9. LUCENE-1593: MultiSearcher and ParallelMultiSearcher did not break ties (in 
+   sort) by doc Id in a consistent manner (i.e., if Sort.FIELD_DOC was used vs. 
+   when it wasn't). (Shai Erera via Michael McCandless)
+
  New features
 
  1. LUCENE-1411: Added expert API to open an IndexWriter on a prior

Modified: lucene/java/trunk/common-build.xml
URL: http://svn.apache.org/viewvc/lucene/java/trunk/common-build.xml?rev=773194&r1=773193&r2=773194&view=diff
==============================================================================
--- lucene/java/trunk/common-build.xml (original)
+++ lucene/java/trunk/common-build.xml Sat May  9 10:36:41 2009
@@ -41,7 +41,7 @@
   <property name="name" value="${ant.project.name}"/>
   <property name="Name" value="Lucene"/>
   <property name="version" value="2.9-dev"/>
-  <property name="compatibility.tag" value="lucene_2_4_back_compat_tests_20090501a"/>
+  <property name="compatibility.tag" value="lucene_2_4_back_compat_tests_20090509"/>
   <property name="spec.version" value="${version}"/>	
   <property name="year" value="2000-${current.year}"/>
   <property name="final.name" value="lucene-${name}-${version}"/>

Modified: lucene/java/trunk/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ReadTask.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ReadTask.java?rev=773194&r1=773193&r2=773194&view=diff
==============================================================================
--- lucene/java/trunk/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ReadTask.java (original)
+++ lucene/java/trunk/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ReadTask.java Sat May  9 10:36:41 2009
@@ -102,8 +102,10 @@
       final int numHits = numHits();
       if (numHits > 0) {
         if (sort != null) {
+          // TODO: change the following to create TFC with in/out-of order
+          // according to whether the query's Scorer.
           TopFieldCollector collector = TopFieldCollector.create(sort, numHits,
-              true, withScore(), withMaxScore());
+              true, withScore(), withMaxScore(), false);
           searcher.search(q, collector);
           hits = collector.topDocs();
         } else {

Modified: lucene/java/trunk/src/demo/org/apache/lucene/demo/SearchFiles.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/demo/org/apache/lucene/demo/SearchFiles.java?rev=773194&r1=773193&r2=773194&view=diff
==============================================================================
--- lucene/java/trunk/src/demo/org/apache/lucene/demo/SearchFiles.java (original)
+++ lucene/java/trunk/src/demo/org/apache/lucene/demo/SearchFiles.java Sat May  9 10:36:41 2009
@@ -193,7 +193,8 @@
                                      int hitsPerPage, boolean raw, boolean interactive) throws IOException {
  
     // Collect enough docs to show 5 pages
-    TopScoreDocCollector collector = new TopScoreDocCollector(5 * hitsPerPage);
+    TopScoreDocCollector collector = TopScoreDocCollector.create(
+        5 * hitsPerPage, false);
     searcher.search(query, collector);
     ScoreDoc[] hits = collector.topDocs().scoreDocs;
     
@@ -212,7 +213,7 @@
           break;
         }
 
-        collector = new TopScoreDocCollector(numTotalHits);
+        collector = TopScoreDocCollector.create(numTotalHits, false);
         searcher.search(query, collector);
         hits = collector.topDocs().scoreDocs;
       }

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/BooleanQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/BooleanQuery.java?rev=773194&r1=773193&r2=773194&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/BooleanQuery.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/BooleanQuery.java Sat May  9 10:36:41 2009
@@ -221,6 +221,9 @@
      *          and scores documents in document number order.
      */
     public Scorer scorer(IndexReader reader) throws IOException {
+      // TODO (3.0): instantiate either BS or BS2, according to
+      // allowDocsOutOfOrder (basically, try to inline BS2.score(Collector)'s
+      // logic.
       BooleanScorer2 result = new BooleanScorer2(similarity,
                                                  minNrShouldMatch,
                                                  allowDocsOutOfOrder);
@@ -474,5 +477,5 @@
     return Float.floatToIntBits(getBoost()) ^ clauses.hashCode()
            + getMinimumNumberShouldMatch();
   }
-
+  
 }

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/BooleanScorer.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/BooleanScorer.java?rev=773194&r1=773193&r2=773194&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/BooleanScorer.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/BooleanScorer.java Sat May  9 10:36:41 2009
@@ -99,21 +99,23 @@
     throws IOException {
     int mask = 0;
     if (required || prohibited) {
-      if (nextMask == 0)
-        throw new IndexOutOfBoundsException
-          ("More than 32 required/prohibited clauses in query.");
+      if (nextMask == 0) {
+        throw new IndexOutOfBoundsException(
+            "More than 32 required/prohibited clauses in query.");
+      }
       mask = nextMask;
       nextMask = nextMask << 1;
-    } else
-      mask = 0;
+    }
 
-    if (!prohibited)
+    if (!prohibited) {
       maxCoord++;
-
-    if (prohibited)
+      if (required) {
+        requiredMask |= mask;                       // update required mask
+      }
+    } else {
+      // prohibited
       prohibitedMask |= mask;                     // update prohibited mask
-    else if (required)
-      requiredMask |= mask;                       // update required mask
+    }
 
     scorers = new SubScorer(scorer, required, prohibited,
                             bucketTable.newCollector(mask), scorers);
@@ -121,8 +123,10 @@
 
   private final void computeCoordFactors() {
     coordFactors = new float[maxCoord];
-    for (int i = 0; i < maxCoord; i++)
-      coordFactors[i] = getSimilarity().coord(i, maxCoord-1);
+    Similarity sim = getSimilarity();
+    for (int i = 0; i < maxCoord; i++) {
+      coordFactors[i] = sim.coord(i, maxCoord - 1); 
+    }
   }
 
   private int end;
@@ -130,8 +134,7 @@
 
   /** @deprecated use {@link #score(Collector)} instead. */
   public void score(HitCollector hc) throws IOException {
-    next();
-    score(hc, Integer.MAX_VALUE);
+    score(new HitCollectorWrapper(hc));
   }
   
   public void score(Collector collector) throws IOException {
@@ -145,9 +148,9 @@
   }
 
   protected boolean score(Collector collector, int max) throws IOException {
-    if (coordFactors == null)
+    if (coordFactors == null) {
       computeCoordFactors();
-
+    }
     boolean more;
     Bucket tmp;
     
@@ -241,8 +244,9 @@
   }
 
   public float score() {
-    if (coordFactors == null)
+    if (coordFactors == null) {
       computeCoordFactors();
+    }
     return current.score * coordFactors[current.coord];
   }
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/BooleanScorer2.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/BooleanScorer2.java?rev=773194&r1=773193&r2=773194&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/BooleanScorer2.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/BooleanScorer2.java Sat May  9 10:36:41 2009
@@ -36,9 +36,9 @@
   private ArrayList prohibitedScorers = new ArrayList();
 
   private class Coordinator {
+    float[] coordFactors = null;
     int maxCoord = 0; // to be increased for each non prohibited scorer
-    
-    private float[] coordFactors = null;
+    int nrMatchers; // to be increased by score() of match counting scorers.
     
     void init() { // use after all scorers have been added.
       coordFactors = new float[maxCoord + 1];
@@ -48,15 +48,6 @@
       }
     }
     
-    int nrMatchers; // to be increased by score() of match counting scorers.
-
-    void initDoc() {
-      nrMatchers = 0;
-    }
-    
-    float coordFactor() {
-      return coordFactors[nrMatchers];
-    }
   }
 
   private final Coordinator coordinator;
@@ -85,7 +76,7 @@
    * @param allowDocsOutOfOrder Whether it is allowed to return documents out of order.
    *                            This can accelerate the scoring of disjunction queries.                         
    */
-  public BooleanScorer2(Similarity similarity, int minNrShouldMatch, boolean allowDocsOutOfOrder) {
+  public BooleanScorer2(Similarity similarity, int minNrShouldMatch, boolean allowDocsOutOfOrder) throws IOException {
     super(similarity);
     if (minNrShouldMatch < 0) {
       throw new IllegalArgumentException("Minimum number of optional scorers should not be negative");
@@ -105,7 +96,7 @@
    *                         at least one of the optional scorers will have to
    *                         match during the search.
    */
-  public BooleanScorer2(Similarity similarity, int minNrShouldMatch) {
+  public BooleanScorer2(Similarity similarity, int minNrShouldMatch) throws IOException {
     this(similarity, minNrShouldMatch, false);
   }
   
@@ -114,11 +105,11 @@
    *  at least one of the optional scorers will have to match during the search.
    * @param similarity The similarity to be used.
    */
-  public BooleanScorer2(Similarity similarity) {
+  public BooleanScorer2(Similarity similarity) throws IOException {
     this(similarity, 0, false);
   }
 
-  public void add(final Scorer scorer, boolean required, boolean prohibited) {
+  public void add(final Scorer scorer, boolean required, boolean prohibited) throws IOException {
     if (!prohibited) {
       coordinator.maxCoord++;
     }
@@ -151,17 +142,24 @@
   private class SingleMatchScorer extends Scorer {
     private Scorer scorer;
     private int lastScoredDoc = -1;
+    // Save the score of lastScoredDoc, so that we don't compute it more than
+    // once in score().
+    private float lastDocScore = Float.NaN;
 
     SingleMatchScorer(Scorer scorer) {
       super(scorer.getSimilarity());
       this.scorer = scorer;
     }
     public float score() throws IOException {
-      if (this.doc() >= lastScoredDoc) {
-        lastScoredDoc = this.doc();
+      int doc = doc();
+      if (doc >= lastScoredDoc) {
+        if (doc > lastScoredDoc) {
+          lastDocScore = scorer.score();
+          lastScoredDoc = doc;
+        }
         coordinator.nrMatchers++;
       }
-      return scorer.score();
+      return lastDocScore;
     }
     public int doc() {
       return scorer.doc();
@@ -178,39 +176,51 @@
   }
 
   private Scorer countingDisjunctionSumScorer(final List scorers,
-                                              int minNrShouldMatch) throws IOException
-  // each scorer from the list counted as a single matcher
-  {
+      int minNrShouldMatch) throws IOException {
+    // each scorer from the list counted as a single matcher
     return new DisjunctionSumScorer(scorers, minNrShouldMatch) {
       private int lastScoredDoc = -1;
+      // Save the score of lastScoredDoc, so that we don't compute it more than
+      // once in score().
+      private float lastDocScore = Float.NaN;
       public float score() throws IOException {
-        if (this.doc() >= lastScoredDoc) {
-          lastScoredDoc = this.doc();
+        int doc = doc();
+        if (doc >= lastScoredDoc) {
+          if (doc > lastScoredDoc) {
+            lastDocScore = super.score();
+            lastScoredDoc = doc;
+          }
           coordinator.nrMatchers += super.nrMatchers;
         }
-        return super.score();
+        return lastDocScore;
       }
     };
   }
 
-  private static Similarity defaultSimilarity = new DefaultSimilarity();
+  private static final Similarity defaultSimilarity = Similarity.getDefault();
 
   private Scorer countingConjunctionSumScorer(List requiredScorers) throws IOException {
     // each scorer from the list counted as a single matcher
     final int requiredNrMatchers = requiredScorers.size();
     return new ConjunctionScorer(defaultSimilarity, requiredScorers) {
       private int lastScoredDoc = -1;
-
+      // Save the score of lastScoredDoc, so that we don't compute it more than
+      // once in score().
+      private float lastDocScore = Float.NaN;
       public float score() throws IOException {
-        if (this.doc() >= lastScoredDoc) {
-          lastScoredDoc = this.doc();
+        int doc = doc();
+        if (doc >= lastScoredDoc) {
+          if (doc > lastScoredDoc) {
+            lastDocScore = super.score();
+            lastScoredDoc = doc;
+          }
           coordinator.nrMatchers += requiredNrMatchers;
         }
         // All scorers match, so defaultSimilarity super.score() always has 1 as
         // the coordination factor.
         // Therefore the sum of the scores of the requiredScorers
         // is used as score.
-        return super.score();
+        return lastDocScore;
       }
     };
   }
@@ -361,7 +371,7 @@
     collector.setScorer(this);
     while (docNr < max) {
       collector.collect(docNr);
-      if (! countingSumScorer.next()) {
+      if (!countingSumScorer.next()) {
         return false;
       }
       docNr = countingSumScorer.doc();
@@ -379,9 +389,9 @@
   }
 
   public float score() throws IOException {
-    coordinator.initDoc();
+    coordinator.nrMatchers = 0;
     float sum = countingSumScorer.score();
-    return sum * coordinator.coordFactor();
+    return sum * coordinator.coordFactors[coordinator.nrMatchers];
   }
 
   /** Skips to the first match beyond the current whose document number is

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/FieldDoc.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/FieldDoc.java?rev=773194&r1=773193&r2=773194&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/FieldDoc.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/FieldDoc.java Sat May  9 10:36:41 2009
@@ -17,7 +17,6 @@
  * limitations under the License.
  */
 
-
 /**
  * Expert: A ScoreDoc which also contains information about
  * how to sort the referenced document.  In addition to the
@@ -38,8 +37,7 @@
  * @see ScoreDoc
  * @see TopFieldDocs
  */
-public class FieldDoc
-extends ScoreDoc {
+public class FieldDoc extends ScoreDoc {
 
 	/** Expert: The values which are used to sort the referenced document.
 	 * The order of these will match the original sort criteria given by a
@@ -60,4 +58,18 @@
 		super (doc, score);
 		this.fields = fields;
 	}
-}
\ No newline at end of file
+	
+	// A convenience method for debugging.
+	public String toString() {
+	  // super.toString returns the doc and score information, so just add the
+          // fields information
+	  StringBuffer sb = new StringBuffer(super.toString());
+	  sb.append("[");
+	  for (int i = 0; i < fields.length; i++) {
+            sb.append(fields[i]).append(", ");
+          }
+	  sb.setLength(sb.length() - 2); // discard last ", "
+	  sb.append("]");
+	  return super.toString();
+	}
+}

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/HitQueue.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/HitQueue.java?rev=773194&r1=773193&r2=773194&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/HitQueue.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/HitQueue.java Sat May  9 10:36:41 2009
@@ -20,10 +20,58 @@
 import org.apache.lucene.util.PriorityQueue;
 
 final class HitQueue extends PriorityQueue {
-  HitQueue(int size) {
+
+  private boolean prePopulate;
+
+  /**
+   * Creates a new instance with <code>size</code> elements. If
+   * <code>prePopulate</code> is set to true, the queue will pre-populate itself
+   * with sentinel objects and set its {@link #size()} to <code>size</code>. In
+   * that case, you should not rely on {@link #size()} to get the number of
+   * actual elements that were added to the queue, but keep track yourself.<br>
+   * <b>NOTE:</b> in case <code>prePopulate</code> is true, you should pop
+   * elements from the queue using the following code example:
+   * 
+   * <pre>
+   * PriorityQueue pq = new HitQueue(10, true); // pre-populate.
+   * ScoreDoc top = pq.top();
+   * 
+   * // Add/Update one element.
+   * top.score = 1.0f;
+   * top.doc = 0;
+   * top = (ScoreDoc) pq.updateTop();
+   * int totalHits = 1;
+   * 
+   * // Now pop only the elements that were *truly* inserted.
+   * // First, pop all the sentinel elements (there are pq.size() - totalHits).
+   * for (int i = pq.size() - totalHits; i &gt; 0; i--) pq.pop();
+   * 
+   * // Now pop the truly added elements.
+   * ScoreDoc[] results = new ScoreDoc[totalHits];
+   * for (int i = totalHits - 1; i &gt;= 0; i--) {
+   *   results[i] = (ScoreDoc) pq.pop();
+   * }
+   * </pre>
+   * 
+   * @param size
+   *          the requested size of this queue.
+   * @param prePopulate
+   *          specifies whether to pre-populate the queue with sentinel values.
+   * @see #getSentinelObject()
+   */
+  HitQueue(int size, boolean prePopulate) {
+    this.prePopulate = prePopulate;
     initialize(size);
   }
 
+  // Returns null if prePopulate is false.
+  protected Object getSentinelObject() {
+    // Always set the doc Id to MAX_VALUE so that it won't be favored by
+    // lessThan. This generally should not happen since if score is not NEG_INF,
+    // TopScoreDocCollector will always add the object to the queue.
+    return !prePopulate ? null : new ScoreDoc(Integer.MAX_VALUE, Float.NEGATIVE_INFINITY);
+  }
+  
   protected final boolean lessThan(Object a, Object b) {
     ScoreDoc hitA = (ScoreDoc)a;
     ScoreDoc hitB = (ScoreDoc)b;

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/IndexSearcher.java?rev=773194&r1=773193&r2=773194&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/IndexSearcher.java Sat May  9 10:36:41 2009
@@ -18,9 +18,9 @@
  */
 
 import java.io.IOException;
-import java.util.List;
 import java.util.ArrayList;
-import org.apache.lucene.util.SorterTemplate;
+import java.util.List;
+
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.FieldSelector;
 import org.apache.lucene.index.CorruptIndexException;
@@ -40,15 +40,15 @@
 public class IndexSearcher extends Searcher {
   IndexReader reader;
   private boolean closeReader;
-  private IndexReader[] sortedSubReaders;
-  private int[] sortedStarts;
+  private IndexReader[] subReaders;
+  private int[] docStarts;
 
   /** Creates a searcher searching the index in the named directory.
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
    */
   public IndexSearcher(String path) throws CorruptIndexException, IOException {
-    this(IndexReader.open(path), true, false);
+    this(IndexReader.open(path), true);
   }
 
   /** Creates a searcher searching the index in the provided directory.
@@ -56,28 +56,27 @@
    * @throws IOException if there is a low-level IO error
    */
   public IndexSearcher(Directory directory) throws CorruptIndexException, IOException {
-    this(IndexReader.open(directory), true, false);
+    this(IndexReader.open(directory), true);
   }
 
   /** Creates a searcher searching the provided index. */
   public IndexSearcher(IndexReader r) {
-    this(r, false, false);
+    this(r, false);
   }
   
-  /** Expert: Creates a searcher searching the provided
-   *  index, specifying whether searches must visit the
-   *  documents in order.  By default, segments are searched
-   *  in order of decreasing numDocs(); if you pass true for
-   *  docsInOrder, they will instead be searched in their
-   *  natural (unsorted) order.*/
-  public IndexSearcher(IndexReader r, boolean docsInOrder) {
-    this(r, false, docsInOrder);
-  }
-  
-  private IndexSearcher(IndexReader r, boolean closeReader, boolean docsInOrder) {
+  private IndexSearcher(IndexReader r, boolean closeReader) {
     reader = r;
     this.closeReader = closeReader;
-    sortSubReaders(docsInOrder);
+
+    List subReadersList = new ArrayList();
+    gatherSubReaders(subReadersList, reader);
+    subReaders = (IndexReader[]) subReadersList.toArray(new IndexReader[subReadersList.size()]);
+    docStarts = new int[subReaders.length];
+    int maxDoc = 0;
+    for (int i = 0; i < subReaders.length; i++) {
+      docStarts[i] = maxDoc;
+      maxDoc += subReaders[i].maxDoc();
+    }
   }
 
   protected void gatherSubReaders(List allSubReaders, IndexReader r) {
@@ -86,54 +85,12 @@
       // Add the reader itself, and do not recurse
       allSubReaders.add(r);
     } else {
-      for(int i=0;i<subReaders.length;i++) {
+      for (int i = 0; i < subReaders.length; i++) {
         gatherSubReaders(allSubReaders, subReaders[i]);
       }
     }
   }
 
-  static private final IndexReader[] indexReaderZeroArray = new IndexReader[0];
-
-  protected void sortSubReaders(boolean docsInOrder) {
-
-    List subReadersList = new ArrayList();
-    gatherSubReaders(subReadersList, reader);
-    sortedSubReaders = (IndexReader[]) subReadersList.toArray(indexReaderZeroArray);
-    final int length = sortedSubReaders.length;
-    sortedStarts = new int[length];
-    int maxDoc = 0;
-    for (int i = 0; i < sortedSubReaders.length; i++) {
-      sortedStarts[i] = maxDoc;
-      maxDoc += sortedSubReaders[i].maxDoc();          // compute maxDocs
-    }
-
-    if (!docsInOrder) {
-
-      // sort readers and starts
-      SorterTemplate sorter = new SorterTemplate() {
-          protected int compare(int i, int j) {
-            int num1 = sortedSubReaders[i].numDocs();
-            int num2 = sortedSubReaders[j].numDocs();
-            if (num1 > num2)
-              return -1;
-            if (num1 < num2)
-              return 1;
-            return 0;
-          }
-          protected void swap(int i, int j) {
-            IndexReader temp = sortedSubReaders[i];
-            sortedSubReaders[i] = sortedSubReaders[j];
-            sortedSubReaders[j] = temp;
-
-            int tempInt = sortedStarts[i];
-            sortedStarts[i] = sortedStarts[j];
-            sortedStarts[j] = tempInt;
-          }
-        };
-      sorter.quickSort(0, length - 1);
-    }
-  }
-  
   /** Return the {@link IndexReader} this searches. */
   public IndexReader getIndexReader() {
     return reader;
@@ -177,7 +134,10 @@
     if (nDocs <= 0)  // null might be returned from hq.top() below.
       throw new IllegalArgumentException("nDocs must be > 0");
 
-    TopScoreDocCollector collector = new TopScoreDocCollector(nDocs);
+    // TODO: The following should be changed to first obtain a Scorer and then ask it
+    // if it's going to return in-order or out-of-order docs, and create TSDC
+    // accordingly.
+    TopScoreDocCollector collector = TopScoreDocCollector.create(nDocs, false);
     search(weight, filter, collector);
     return collector.topDocs();
   }
@@ -233,10 +193,14 @@
       return (TopFieldDocs) collector.topDocs();
     }
     // Search each sub-reader
-    // TODO: by default we should create a TopFieldCollector which does not
-    // track document scores and maxScore. Currently the default is set to true,
-    // however it will change in 3.0.
-    TopFieldCollector collector = TopFieldCollector.create(sort, nDocs, fillFields, true, true);
+    // TODO (3.0): by default we should create a TopFieldCollector which does
+    // not track document scores and maxScore. Currently the default is set to
+    // true, however it will change in 3.0.
+    // TODO: The following should be changed to first obtain a Scorer and then ask it
+    // if it's going to return in-order or out-of-order docs, and create TSDC
+    // accordingly.
+    TopFieldCollector collector = TopFieldCollector.create(sort, nDocs,
+        fillFields, true, true, false);
     search(weight, filter, collector);
     return (TopFieldDocs) collector.topDocs();
   }
@@ -252,9 +216,9 @@
   public void search(Weight weight, Filter filter, Collector collector)
       throws IOException {
     
-    for (int i = 0; i < sortedSubReaders.length; i++) { // search each subreader
-      collector.setNextReader(sortedSubReaders[i], sortedStarts[i]);
-      doSearch(sortedSubReaders[i], weight, filter, collector);
+    for (int i = 0; i < subReaders.length; i++) { // search each subreader
+      collector.setNextReader(subReaders[i], docStarts[i]);
+      doSearch(subReaders[i], weight, filter, collector);
     }
   }
   

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/MultiSearcher.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/MultiSearcher.java?rev=773194&r1=773193&r2=773194&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/MultiSearcher.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/MultiSearcher.java Sat May  9 10:36:41 2009
@@ -203,7 +203,7 @@
   public TopDocs search(Weight weight, Filter filter, int nDocs)
   throws IOException {
 
-    HitQueue hq = new HitQueue(nDocs);
+    HitQueue hq = new HitQueue(nDocs, false);
     int totalHits = 0;
 
     for (int i = 0; i < searchables.length; i++) { // search each searcher
@@ -236,7 +236,19 @@
     
     for (int i = 0; i < searchables.length; i++) { // search each searcher
       TopFieldDocs docs = searchables[i].search (weight, filter, n, sort);
-      
+      // If one of the Sort fields is FIELD_DOC, need to fix its values, so that
+      // it will break ties by doc Id properly. Otherwise, it will compare to
+      // 'relative' doc Ids, that belong to two different searchers.
+      for (int j = 0; j < docs.fields.length; j++) {
+        if (docs.fields[j].getType() == SortField.DOC) {
+          // iterate over the score docs and change their fields value
+          for (int j2 = 0; j2 < docs.scoreDocs.length; j2++) {
+            FieldDoc fd = (FieldDoc) docs.scoreDocs[j2];
+            fd.fields[j] = new Integer(((Integer) fd.fields[j]).intValue() + starts[i]);
+          }
+          break;
+        }
+      }
       if (hq == null) hq = new FieldDocSortedHitQueue (docs.fields, n);
       totalHits += docs.totalHits;		  // update totalHits
       maxScore = Math.max(maxScore, docs.getMaxScore());

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/ParallelMultiSearcher.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/ParallelMultiSearcher.java?rev=773194&r1=773193&r2=773194&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/ParallelMultiSearcher.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/ParallelMultiSearcher.java Sat May  9 10:36:41 2009
@@ -54,7 +54,7 @@
    */
   public TopDocs search(Weight weight, Filter filter, int nDocs)
     throws IOException {
-    HitQueue hq = new HitQueue(nDocs);
+    HitQueue hq = new HitQueue(nDocs, false);
     int totalHits = 0;
     MultiSearcherThread[] msta =
       new MultiSearcherThread[searchables.length];
@@ -295,7 +295,22 @@
       // the actual type of fields, in case the original list contained AUTO.
       // if the searchable returns null for fields, we'll have problems.
       if (sort != null) {
-        ((FieldDocSortedHitQueue)hq).setFields (((TopFieldDocs)docs).fields);
+        TopFieldDocs docsFields = (TopFieldDocs) docs;
+        // If one of the Sort fields is FIELD_DOC, need to fix its values, so that
+        // it will break ties by doc Id properly. Otherwise, it will compare to
+        // 'relative' doc Ids, that belong to two different searchers.
+        for (int j = 0; j < docsFields.fields.length; j++) {
+          if (docsFields.fields[j].getType() == SortField.DOC) {
+            // iterate over the score docs and change their fields value
+            for (int j2 = 0; j2 < docs.scoreDocs.length; j2++) {
+              FieldDoc fd = (FieldDoc) docs.scoreDocs[j2];
+              fd.fields[j] = new Integer(((Integer) fd.fields[j]).intValue() + starts[i]);
+            }
+            break;
+          }
+        }
+
+        ((FieldDocSortedHitQueue) hq).setFields(docsFields.fields);
       }
       ScoreDoc[] scoreDocs = docs.scoreDocs;
       for (int j = 0;

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/ScoreDoc.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/ScoreDoc.java?rev=773194&r1=773193&r2=773194&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/ScoreDoc.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/ScoreDoc.java Sat May  9 10:36:41 2009
@@ -33,4 +33,10 @@
     this.doc = doc;
     this.score = score;
   }
+  
+  // A convenience method for debugging.
+  public String toString() {
+    return "doc=" + doc + " score=" + score;
+  }
+  
 }

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/Sort.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/Sort.java?rev=773194&r1=773193&r2=773194&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/Sort.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/Sort.java Sat May  9 10:36:41 2009
@@ -121,7 +121,7 @@
    * only with slightly more overhead.
    */
   public Sort() {
-    this(new SortField[] { SortField.FIELD_SCORE, SortField.FIELD_DOC });
+    this(SortField.FIELD_SCORE);
   }
 
   /**
@@ -179,9 +179,7 @@
    * then by index order (document number).
    */
   public void setSort(String field, boolean reverse) {
-    SortField[] nfields = new SortField[] {
-        new SortField(field, SortField.AUTO, reverse), SortField.FIELD_DOC };
-    fields = nfields;
+    fields = new SortField[] { new SortField(field, SortField.AUTO, reverse) };
   }
 
   /** Sets the sort to the terms in each field in succession. */

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/TopDocCollector.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/TopDocCollector.java?rev=773194&r1=773193&r2=773194&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/TopDocCollector.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/TopDocCollector.java Sat May  9 10:36:41 2009
@@ -44,7 +44,7 @@
    * @param numHits the maximum number of hits to collect
    */
   public TopDocCollector(int numHits) {
-    this(new HitQueue(numHits));
+    this(new HitQueue(numHits, false));
   }
 
   /** @deprecated use TopDocCollector(hq) instead. numHits is not used by this

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/TopDocsCollector.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/TopDocsCollector.java?rev=773194&r1=773193&r2=773194&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/TopDocsCollector.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/TopDocsCollector.java Sat May  9 10:36:41 2009
@@ -75,7 +75,10 @@
   
   /** Returns the top docs that were collected by this collector. */
   public final TopDocs topDocs() {
-    return topDocs(0, pq.size());
+    // In case pq was populated with sentinel values, there might be less
+    // results than pq.size(). Therefore return all results until either
+    // pq.size() or totalHits.
+    return topDocs(0, totalHits < pq.size() ? totalHits : pq.size());
   }
 
   /**
@@ -91,7 +94,10 @@
    * results this search execution collected.
    */
   public final TopDocs topDocs(int start) {
-    return topDocs(start, pq.size());
+    // In case pq was populated with sentinel values, there might be less
+    // results than pq.size(). Therefore return all results until either
+    // pq.size() or totalHits.
+    return topDocs(start, totalHits < pq.size() ? totalHits : pq.size());
   }
 
   /**
@@ -108,18 +114,21 @@
    * returned {@link TopDocs} object, which will contain all the results this
    * search execution collected.
    */
-  public TopDocs topDocs(int start, int howMany) {
+  public final TopDocs topDocs(int start, int howMany) {
     
-    int pqsize = pq.size();
+    // In case pq was populated with sentinel values, there might be less
+    // results than pq.size(). Therefore return all results until either
+    // pq.size() or totalHits.
+    int size = totalHits < pq.size() ? totalHits : pq.size();
 
     // Don't bother to throw an exception, just return an empty TopDocs in case
     // the parameters are invalid or out of range.
-    if (start < 0 || start >= pqsize || howMany <= 0) {
+    if (start < 0 || start >= size || howMany <= 0) {
       return newTopDocs(null, start);
     }
 
     // We know that start < pqsize, so just fix howMany. 
-    howMany = Math.min(pqsize - start, howMany);
+    howMany = Math.min(size - start, howMany);
     ScoreDoc[] results = new ScoreDoc[howMany];
 
     // pq's pop() returns the 'least' element in the queue, therefore need
@@ -127,7 +136,7 @@
     // Note that this loop will usually not be executed, since the common usage
     // should be that the caller asks for the last howMany results. However it's
     // needed here for completeness.
-    for (int i = pqsize - start - howMany; i > 0; i--) { pq.pop(); }
+    for (int i = pq.size() - start - howMany; i > 0; i--) { pq.pop(); }
     
     // Get the requested results from pq.
     populateResults(results, howMany);

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/TopFieldCollector.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/TopFieldCollector.java?rev=773194&r1=773193&r2=773194&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/TopFieldCollector.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/TopFieldCollector.java Sat May  9 10:36:41 2009
@@ -41,7 +41,7 @@
    * Implements a TopFieldCollector over one SortField criteria, without
    * tracking document scores and maxScore.
    */
-  private static class OneComparatorNonScoringCollector extends
+  private static class OneComparatorNonScoringCollector extends 
       TopFieldCollector {
 
     final FieldComparator comparator;
@@ -54,19 +54,19 @@
       reverseMul = queue.getReverseMul()[0];
     }
     
-    private final void updateBottom(int doc) {
+    final void updateBottom(int doc) {
       // bottom.score is already set to Float.NaN in add().
       bottom.docID = docBase + doc;
-      pq.adjustTop();
-      bottom = (FieldValueHitQueue.Entry) pq.top();
+      bottom = (Entry) pq.updateTop();
     }
 
     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.docID)) {
+        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;
         }
         
@@ -99,32 +99,71 @@
   }
 
   /*
+   * 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 queue,
+        int numHits, boolean fillFields) throws IOException {
+      super(queue, numHits, fillFields);
+    }
+    
+    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.docID)) {
+          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);
+        }
+      }
+    }
+  }
+
+  /*
    * Implements a TopFieldCollector over one SortField criteria, while tracking
    * document scores but no maxScore.
    */
   private static class OneComparatorScoringNoMaxScoreCollector extends
       OneComparatorNonScoringCollector {
 
-    private Scorer scorer;
+    Scorer scorer;
 
     public OneComparatorScoringNoMaxScoreCollector(FieldValueHitQueue queue,
         int numHits, boolean fillFields) throws IOException {
       super(queue, numHits, fillFields);
     }
     
-    private final void updateBottom(int doc, float score) {
+    final void updateBottom(int doc, float score) {
       bottom.docID = docBase + doc;
       bottom.score = score;
-      pq.adjustTop();
-      bottom = (FieldValueHitQueue.Entry) pq.top();
+      bottom = (Entry) pq.updateTop();
     }
 
     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.docID)) {
+        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;
         }
         
@@ -150,12 +189,6 @@
       }
     }
     
-    public void setNextReader(IndexReader reader, int docBase) throws IOException {
-      final int numSlotsFull = queueFull ? numHits : totalHits;
-      this.docBase = docBase;
-      comparator.setNextReader(reader, docBase, numSlotsFull);
-    }
-    
     public void setScorer(Scorer scorer) throws IOException {
       this.scorer = scorer;
       comparator.setScorer(scorer);
@@ -164,13 +197,59 @@
   }
 
   /*
+   * 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 queue, int numHits, boolean fillFields)
+        throws IOException {
+      super(queue, numHits, fillFields);
+    }
+    
+    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.docID)) {
+          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);
+        }
+      }
+    }
+  }
+
+  /*
    * Implements a TopFieldCollector over one SortField criteria, with tracking
    * document scores and maxScore.
    */
-  private final static class OneComparatorScoringMaxScoreCollector extends
+  private static class OneComparatorScoringMaxScoreCollector extends
       OneComparatorNonScoringCollector {
 
-    private Scorer scorer;
+    Scorer scorer;
     
     public OneComparatorScoringMaxScoreCollector(FieldValueHitQueue queue,
         int numHits, boolean fillFields) throws IOException {
@@ -179,11 +258,10 @@
       maxScore = Float.NEGATIVE_INFINITY;
     }
     
-    private final void updateBottom(int doc, float score) {
+    final void updateBottom(int doc, float score) {
       bottom.docID = docBase + doc;
       bottom.score = score;
-      pq.adjustTop();
-      bottom = (FieldValueHitQueue.Entry) pq.top();
+      bottom = (Entry) pq.updateTop();
     }
 
     public void collect(int doc) throws IOException {
@@ -193,9 +271,10 @@
       }
       ++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.docID)) {
+        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;
         }
         
@@ -223,6 +302,50 @@
   }
 
   /*
+   * 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 queue,
+        int numHits, boolean fillFields) throws IOException {
+      super(queue, numHits, fillFields);
+    }
+    
+    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.docID)) {
+          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);
+        }
+      }
+
+    }
+  }
+
+  /*
    * Implements a TopFieldCollector over multiple SortField criteria, without
    * tracking document scores and maxScore.
    */
@@ -238,11 +361,10 @@
       reverseMul = queue.getReverseMul();
     }
     
-    private final void updateBottom(int doc) {
+    final void updateBottom(int doc) {
       // bottom.score is already set to Float.NaN in add().
       bottom.docID = docBase + doc;
-      pq.adjustTop();
-      bottom = (FieldValueHitQueue.Entry) pq.top();
+      bottom = (Entry) pq.updateTop();
     }
 
     public void collect(int doc) throws IOException {
@@ -252,18 +374,16 @@
         for (int i = 0;; i++) {
           final int c = reverseMul[i] * comparators[i].compareBottom(doc);
           if (c < 0) {
-            // Definitely not competitive
+            // Definitely not competitive.
             return;
           } else if (c > 0) {
-            // Definitely competitive
+            // Definitely competitive.
             break;
           } else if (i == comparators.length - 1) {
-            // This is the equals case.
-            if (doc + docBase > bottom.docID) {
-              // Definitely not competitive
-              return;
-            }
-            break;
+            // 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;
           }
         }
 
@@ -310,12 +430,74 @@
   }
   
   /*
+   * 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 queue,
+        int numHits, boolean fillFields) throws IOException {
+      super(queue, numHits, fillFields);
+    }
+    
+    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.docID) {
+              // Definitely not competitive
+              return;
+            }
+            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);
+          }
+        }
+      }
+    }
+  }
+
+  /*
    * Implements a TopFieldCollector over multiple SortField criteria, with
    * tracking document scores and maxScore.
    */
-  private final static class MultiComparatorScoringMaxScoreCollector extends MultiComparatorNonScoringCollector {
+  private static class MultiComparatorScoringMaxScoreCollector extends MultiComparatorNonScoringCollector {
     
-    private Scorer scorer;
+    Scorer scorer;
     
     public MultiComparatorScoringMaxScoreCollector(FieldValueHitQueue queue,
         int numHits, boolean fillFields) throws IOException {
@@ -324,11 +506,10 @@
       maxScore = Float.NEGATIVE_INFINITY;
     }
     
-    private final void updateBottom(int doc, float score) {
+    final void updateBottom(int doc, float score) {
       bottom.docID = docBase + doc;
       bottom.score = score;
-      pq.adjustTop();
-      bottom = (FieldValueHitQueue.Entry) pq.top();
+      bottom = (Entry) pq.updateTop();
     }
 
     public void collect(int doc) throws IOException {
@@ -342,10 +523,79 @@
         for (int i = 0;; i++) {
           final int c = reverseMul[i] * comparators[i].compareBottom(doc);
           if (c < 0) {
-            // Definitely not competitive
+            // Definitely not competitive.
             return;
           } else if (c > 0) {
-            // Definitely competitive
+            // 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);
+        }
+
+        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);
+          }
+        }
+      }
+    }
+
+    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 OutOfOrderMultiComparatorScoringMaxScoreCollector
+      extends MultiComparatorScoringMaxScoreCollector {
+    
+    public OutOfOrderMultiComparatorScoringMaxScoreCollector(FieldValueHitQueue queue,
+        int numHits, boolean fillFields) throws IOException {
+      super(queue, numHits, fillFields);
+    }
+    
+    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.
@@ -382,31 +632,25 @@
         }
       }
     }
-
-    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.
    */
-  private final static class MultiComparatorScoringNoMaxScoreCollector extends MultiComparatorNonScoringCollector {
+  private static class MultiComparatorScoringNoMaxScoreCollector extends MultiComparatorNonScoringCollector {
     
-    private Scorer scorer;
+    Scorer scorer;
     
     public MultiComparatorScoringNoMaxScoreCollector(FieldValueHitQueue queue,
         int numHits, boolean fillFields) throws IOException {
       super(queue, numHits, fillFields);
     }
     
-    private final void updateBottom(int doc, float score) {
+    final void updateBottom(int doc, float score) {
       bottom.docID = docBase + doc;
       bottom.score = score;
-      pq.adjustTop();
-      bottom = (FieldValueHitQueue.Entry) pq.top();
+      bottom = (Entry) pq.updateTop();
     }
 
     public void collect(int doc) throws IOException {
@@ -416,10 +660,81 @@
         for (int i = 0;; i++) {
           final int c = reverseMul[i] * comparators[i].compareBottom(doc);
           if (c < 0) {
-            // Definitely not competitive
+            // Definitely not competitive.
             return;
           } else if (c > 0) {
-            // Definitely competitive
+            // 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);
+
+        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);
+        }
+
+        // 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);
+          }
+        }
+      }
+    }
+
+    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 queue, int numHits, boolean fillFields)
+        throws IOException {
+      super(queue, numHits, fillFields);
+    }
+    
+    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.
@@ -451,7 +766,7 @@
           comparators[i].copy(slot, doc);
         }
 
-        // Compute score only if it competitive.
+        // Compute score only if it is competitive.
         final float score = scorer.score();
         add(slot, doc, score);
         if (queueFull) {
@@ -518,12 +833,16 @@
    *          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
    */
   public static TopFieldCollector create(Sort sort, int numHits,
-      boolean fillFields, boolean trackDocScores, boolean trackMaxScore)
+      boolean fillFields, boolean trackDocScores, boolean trackMaxScore,
+      boolean docsScoredInOrder)
       throws IOException {
     if (sort.fields.length == 0) {
       throw new IllegalArgumentException("Sort must contain at least one field");
@@ -531,28 +850,47 @@
     
     FieldValueHitQueue queue = FieldValueHitQueue.create(sort.fields, numHits);
     if (queue.getComparators().length == 1) {
-      if (trackMaxScore) {
-        return new OneComparatorScoringMaxScoreCollector(queue, numHits, fillFields);
-      } else if (trackDocScores) {
-        return new OneComparatorScoringNoMaxScoreCollector(queue, numHits, fillFields);
+      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 {
-        return new OneComparatorNonScoringCollector(queue, numHits, fillFields);
+        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 (trackMaxScore) {
-      return new MultiComparatorScoringMaxScoreCollector(queue, numHits, fillFields);
-    } else if (trackDocScores) {
-      return new MultiComparatorScoringNoMaxScoreCollector(queue, numHits, fillFields);
+    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);
+      }
     } else {
-      return new MultiComparatorNonScoringCollector(queue, numHits, fillFields);
+      if (trackMaxScore) {
+        return new OutOfOrderMultiComparatorScoringMaxScoreCollector(queue, numHits, fillFields);
+      } else if (trackDocScores) {
+        return new OutOfOrderMultiComparatorScoringNoMaxScoreCollector(queue, numHits, fillFields);
+      } else {
+        return new OutOfOrderMultiComparatorNonScoringCollector(queue, numHits, fillFields);
+      }
     }
   }
   
   final void add(int slot, int doc, float score) {
-    pq.put(new FieldValueHitQueue.Entry(slot, docBase + doc, score));
-    bottom = (FieldValueHitQueue.Entry) pq.top();
+    bottom = (Entry) pq.add(new Entry(slot, docBase + doc, score));
     queueFull = totalHits == numHits;
   }
 
@@ -562,14 +900,15 @@
    */
 
   protected void populateResults(ScoreDoc[] results, int howMany) {
-    FieldValueHitQueue queue = (FieldValueHitQueue) pq;
     if (fillFields) {
-      for (int i = queue.size() - 1; i >= 0; i--) {
-        results[i] = queue.fillFields((FieldValueHitQueue.Entry) queue.pop());
+      // avoid casting if unnecessary.
+      FieldValueHitQueue queue = (FieldValueHitQueue) pq;
+      for (int i = howMany - 1; i >= 0; i--) {
+        results[i] = queue.fillFields((Entry) queue.pop());
       }
     } else {
-      for (int i = queue.size() - 1; i >= 0; i--) {
-        Entry entry = (FieldValueHitQueue.Entry) queue.pop();
+      for (int i = howMany - 1; i >= 0; i--) {
+        Entry entry = (Entry) pq.pop();
         results[i] = new FieldDoc(entry.docID, entry.score);
       }
     }

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/TopScoreDocCollector.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/TopScoreDocCollector.java?rev=773194&r1=773193&r2=773194&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/TopScoreDocCollector.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/TopScoreDocCollector.java Sat May  9 10:36:41 2009
@@ -22,24 +22,80 @@
 import org.apache.lucene.index.IndexReader;
 
 /**
- * A {@link Collector} implementation that collects the
- * top-scoring hits, returning them as a {@link
- * TopDocs}. This is used by {@link IndexSearcher} to
- * implement {@link TopDocs}-based search.  Hits are sorted
- * by score descending and then (when the scores are tied)
- * docID ascending.
+ * A {@link Collector} implementation that collects the top-scoring hits,
+ * returning them as a {@link TopDocs}. This is used by {@link IndexSearcher} to
+ * implement {@link TopDocs}-based search. Hits are sorted by score descending
+ * and then (when the scores are tied) docID ascending. When you create an
+ * instance of this collector you should know in advance whether documents are
+ * going to be collected in doc Id order or not.
  */
-public final class TopScoreDocCollector extends TopDocsCollector {
+public abstract class TopScoreDocCollector extends TopDocsCollector {
 
-  private ScoreDoc reusableSD;
-  private int docBase = 0;
-  private Scorer scorer;
+  // Assumes docs are scored in order.
+  private static class InOrderTopScoreDocCollector extends TopScoreDocCollector {
+    private InOrderTopScoreDocCollector(int numHits) {
+      super(numHits);
+    }
+    
+    public void collect(int doc) throws IOException {
+      float score = scorer.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 = (ScoreDoc) pq.updateTop();
+    }
+  }
+
+  // Assumes docs are scored out of order.
+  private static class OutOfOrderTopScoreDocCollector extends TopScoreDocCollector {
+    private OutOfOrderTopScoreDocCollector(int numHits) {
+      super(numHits);
+    }
     
-  /** Construct to collect a given number of hits.
-   * @param numHits the maximum number of hits to collect
+    public void collect(int doc) throws IOException {
+      float score = scorer.score();
+      totalHits++;
+      doc += docBase;
+      if (score < pqTop.score || (score == pqTop.score && doc > pqTop.doc)) {
+        return;
+      }
+      pqTop.doc = doc;
+      pqTop.score = score;
+      pqTop = (ScoreDoc) 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)}.
    */
-  public TopScoreDocCollector(int numHits) {
-    super(new HitQueue(numHits));
+  public static TopScoreDocCollector create(int numHits, boolean docsScoredInOrder) {
+    
+    if (docsScoredInOrder) {
+      return new InOrderTopScoreDocCollector(numHits);
+    } else {
+      return new OutOfOrderTopScoreDocCollector(numHits);
+    }
+    
+  }
+  
+  ScoreDoc pqTop;
+  int docBase = 0;
+  Scorer scorer;
+    
+  // prevents instantiation
+  private 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.
+    pqTop = (ScoreDoc) pq.top();
   }
 
   protected TopDocs newTopDocs(ScoreDoc[] results, int start) {
@@ -62,24 +118,6 @@
     return new TopDocs(totalHits, results, maxScore);
   }
   
-  // javadoc inherited
-  public void collect(int doc) throws IOException {
-    float score = scorer.score();
-    totalHits++;
-    if (reusableSD == null) {
-      reusableSD = new ScoreDoc(doc + docBase, score);
-    } else if (score >= reusableSD.score) {
-      // reusableSD holds the last "rejected" entry, so, if
-      // this new score is not better than that, there's no
-      // need to try inserting it
-      reusableSD.doc = doc + docBase;
-      reusableSD.score = score;
-    } else {
-      return;
-    }
-    reusableSD = (ScoreDoc) pq.insertWithOverflow(reusableSD);
-  }
-
   public void setNextReader(IndexReader reader, int base) {
     docBase = base;
   }

Modified: lucene/java/trunk/src/java/org/apache/lucene/util/PriorityQueue.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/util/PriorityQueue.java?rev=773194&r1=773193&r2=773194&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/util/PriorityQueue.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/util/PriorityQueue.java Sat May  9 10:36:41 2009
@@ -29,6 +29,49 @@
     must define this one method. */
   protected abstract boolean lessThan(Object a, Object b);
 
+  /**
+   * This method can be overridden by extending classes to return a sentinel
+   * object which will be used by {@link #initialize(int)} to fill the queue, so
+   * that the code which uses that queue can always assume it's full and only
+   * change the top without attempting to insert any new object.<br>
+   * 
+   * Those sentinel values should always compare worse than any non-sentinel
+   * value (i.e., {@link #lessThan(Object, Object)} should always favor the
+   * non-sentinel values).<br>
+   * 
+   * By default, this method returns false, which means the queue will not be
+   * filled with sentinel values. Otherwise, the value returned will be used to
+   * pre-populate the queue. Adds sentinel values to the queue.<br>
+   * 
+   * If this method is extended to return a non-null value, then the following
+   * usage pattern is recommended:
+   * 
+   * <pre>
+   * // extends getSentinelObject() to return a non-null value.
+   * PriorityQueue pq = new MyQueue(numHits);
+   * // save the 'top' element, which is guaranteed to not be null.
+   * MyObject pqTop = (MyObject) pq.top();
+   * &lt;...&gt;
+   * // now in order to add a new element, which is 'better' than top (after 
+   * // you've verified it is better), it is as simple as:
+   * pqTop.change().
+   * pqTop = pq.updateTop();
+   * </pre>
+   * 
+   * <b>NOTE:</b> if this method returns a non-null value, it will be called by
+   * {@link #initialize(int)} {@link #size()} times, relying on a new object to
+   * be returned and will not check if it's null again. Therefore you should
+   * ensure any call to this method creates a new instance and behaves
+   * consistently, e.g., it cannot return null if it previously returned
+   * non-null.
+   * 
+   * @return the sentinel object to use to pre-populate the queue, or null if
+   *         sentinel objects are not supported.
+   */
+  protected Object getSentinelObject() {
+    return null;
+  }
+
   /** Subclass constructors must call this. */
   protected final void initialize(int maxSize) {
     size = 0;
@@ -40,12 +83,25 @@
       heapSize = maxSize + 1;
     heap = new Object[heapSize];
     this.maxSize = maxSize;
+    
+    // If sentinel objects are supported, populate the queue with them
+    Object sentinel = getSentinelObject();
+    if (sentinel != null) {
+      heap[1] = sentinel;
+      for (int i = 2; i < heap.length; i++) {
+        heap[i] = getSentinelObject();
+      }
+      size = maxSize;
+    }
   }
 
   /**
-   * Adds an Object to a PriorityQueue in log(size) time.
-   * If one tries to add more objects than maxSize from initialize
-   * a RuntimeException (ArrayIndexOutOfBound) is thrown.
+   * Adds an Object to a PriorityQueue in log(size) time. If one tries to add
+   * more objects than maxSize from initialize a RuntimeException
+   * (ArrayIndexOutOfBound) is thrown.
+   * 
+   * @deprecated use {@link #add(Object)} which returns the new top object,
+   *             saving an additional call to {@link #top()}.
    */
   public final void put(Object element) {
     size++;
@@ -54,10 +110,27 @@
   }
 
   /**
-   * Adds element to the PriorityQueue in log(size) time if either
-   * the PriorityQueue is not full, or not lessThan(element, top()).
+   * Adds an Object to a PriorityQueue in log(size) time. If one tries to add
+   * more objects than maxSize from initialize an
+   * {@link ArrayIndexOutOfBoundsException} is thrown.
+   * 
+   * @return the new 'top' element in the queue.
+   */
+  public final Object add(Object element) {
+    size++;
+    heap[size] = element;
+    upHeap();
+    return heap[1];
+  }
+
+  /**
+   * Adds element to the PriorityQueue in log(size) time if either the
+   * PriorityQueue is not full, or not lessThan(element, top()).
+   * 
    * @param element
    * @return true if element is added, false otherwise.
+   * @deprecated use {@link #insertWithOverflow(Object)} instead, which
+   *             encourages objects reuse.
    */
   public boolean insert(Object element) {
     return insertWithOverflow(element) != element;
@@ -109,16 +182,53 @@
       return null;
   }
 
-  /** Should be called when the Object at top changes values.  Still log(n)
-   * worst case, but it's at least twice as fast to <pre>
-   *  { pq.top().change(); pq.adjustTop(); }
-   * </pre> instead of <pre>
-   *  { o = pq.pop(); o.change(); pq.push(o); }
+  /**
+   * Should be called when the Object at top changes values. Still log(n) worst
+   * case, but it's at least twice as fast to
+   * 
+   * <pre>
+   * pq.top().change();
+   * pq.adjustTop();
    * </pre>
+   * 
+   * instead of
+   * 
+   * <pre>
+   * o = pq.pop();
+   * o.change();
+   * pq.push(o);
+   * </pre>
+   * 
+   * @deprecated use {@link #updateTop()} which returns the new top element and
+   *             saves an additional call to {@link #top()}.
    */
   public final void adjustTop() {
     downHeap();
   }
+  
+  /**
+   * Should be called when the Object at top changes values. Still log(n) worst
+   * case, but it's at least twice as fast to
+   * 
+   * <pre>
+   * pq.top().change();
+   * pq.updateTop();
+   * </pre>
+   * 
+   * instead of
+   * 
+   * <pre>
+   * o = pq.pop();
+   * o.change();
+   * pq.push(o);
+   * </pre>
+   * 
+   * @return the new 'top' element.
+   */
+  public final Object updateTop() {
+    downHeap();
+    return heap[1];
+  }
 
   /** Returns the number of elements currently stored in the PriorityQueue. */
   public final int size() {
@@ -127,8 +237,9 @@
 
   /** Removes all entries from the PriorityQueue. */
   public final void clear() {
-    for (int i = 0; i <= size; i++)
+    for (int i = 0; i <= size; i++) {
       heap[i] = null;
+    }
     size = 0;
   }
 

Modified: lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReader.java?rev=773194&r1=773193&r2=773194&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReader.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReader.java Sat May  9 10:36:41 2009
@@ -39,20 +39,17 @@
 import org.apache.lucene.document.Fieldable;
 import org.apache.lucene.document.SetBasedFieldSelector;
 import org.apache.lucene.index.IndexReader.FieldOption;
-import org.apache.lucene.search.Collector;
+import org.apache.lucene.search.FieldCache;
 import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
 import org.apache.lucene.search.ScoreDoc;
-import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.FieldCache;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
 import org.apache.lucene.store.LockObtainFailedException;
 import org.apache.lucene.store.MockRAMDirectory;
-import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.store.NoSuchDirectoryException;
+import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util._TestUtil;
 
@@ -1644,59 +1641,6 @@
     dir.close();
   }
 
-  // LUCENE-1483
-  public void testDocsInOrderSearch() throws Throwable {
-    Directory dir = new MockRAMDirectory();
-    
-    IndexWriter writer = new IndexWriter(dir, new StandardAnalyzer(),
-                                         IndexWriter.MaxFieldLength.LIMITED);
-    writer.addDocument(createDocument("a"));
-    writer.commit();
-    writer.addDocument(createDocument("a"));
-    writer.addDocument(createDocument("a"));
-    writer.close();
-
-    Query q = new TermQuery(new Term("id", "a"));
-
-    IndexSearcher s = new IndexSearcher(dir);
-    s.search(q, new Collector() {
-        int lastDocBase = -1;
-        public void setNextReader(IndexReader reader, int docBase) {
-          if (lastDocBase == -1) {
-            assertEquals(1, docBase);
-          } else if (lastDocBase == 1) {
-            assertEquals(0, docBase);
-          } else {
-            fail();
-          }
-          lastDocBase = docBase;
-        }
-        public void collect(int doc) {}
-        public void setScorer(Scorer scorer) {}
-      });
-    s.close();
-
-    IndexReader r = IndexReader.open(dir);
-    s = new IndexSearcher(r, true);
-    s.search(q, new Collector() {
-        int lastDocBase = -1;
-        public void setNextReader(IndexReader reader, int docBase) {
-          if (lastDocBase == -1) {
-            assertEquals(0, docBase);
-          } else if (lastDocBase == 0) {
-            assertEquals(1, docBase);
-          } else {
-            fail();
-          }
-          lastDocBase = docBase;
-        }
-        public void collect(int doc) {}
-        public void setScorer(Scorer scorer) {}
-      });
-    s.close();
-    r.close();
-  }
-
   // LUCENE-1579: Ensure that on a cloned reader, segments
   // reuse the doc values arrays in FieldCache
   public void testFieldCacheReuseAfterClone() throws Exception {

Modified: lucene/java/trunk/src/test/org/apache/lucene/search/TestPositiveScoresOnlyCollector.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/search/TestPositiveScoresOnlyCollector.java?rev=773194&r1=773193&r2=773194&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/search/TestPositiveScoresOnlyCollector.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/search/TestPositiveScoresOnlyCollector.java Sat May  9 10:36:41 2009
@@ -68,7 +68,7 @@
     }
     
     Scorer s = new SimpleScorer();
-    TopDocsCollector tdc = new TopScoreDocCollector(scores.length);
+    TopDocsCollector tdc = TopScoreDocCollector.create(scores.length, true);
     Collector c = new PositiveScoresOnlyCollector(tdc);
     c.setScorer(s);
     while (!s.next()) {



Mime
View raw message