You are viewing a plain text version of this content. The canonical link for it is here.
Posted to java-commits@lucene.apache.org by mi...@apache.org on 2009/04/13 20:33:58 UTC

svn commit: r764551 [2/3] - in /lucene/java/trunk: ./ contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ contrib/miscellaneous/src/test/org/apache/lucene/index/ contrib/miscellaneous/src/test/org/apache/lucene/misc/ contrib/spatial/sr...

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/Scorer.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/Scorer.java?rev=764551&r1=764550&r2=764551&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/Scorer.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/Scorer.java Mon Apr 13 18:33:56 2009
@@ -52,10 +52,20 @@
    * @param hc The collector to which all matching documents are passed through
    * {@link HitCollector#collect(int, float)}.
    * <br>When this method is used the {@link #explain(int)} method should not be used.
+   * @deprecated use {@link #score(Collector)} instead.
    */
   public void score(HitCollector hc) throws IOException {
+    score(new HitCollectorWrapper(hc));
+  }
+  
+  /** Scores and collects all matching documents.
+   * @param collector The collector to which all matching documents are passed.
+   * <br>When this method is used the {@link #explain(int)} method should not be used.
+   */
+  public void score(Collector collector) throws IOException {
+    collector.setScorer(this);
     while (next()) {
-      hc.collect(doc(), score());
+      collector.collect(doc());
     }
   }
 
@@ -66,10 +76,23 @@
    * {@link HitCollector#collect(int, float)}.
    * @param max Do not score documents past this.
    * @return true if more matching documents may remain.
+   * @deprecated use {@link #score(Collector, int)} instead.
    */
   protected boolean score(HitCollector hc, int max) throws IOException {
+    return score(new HitCollectorWrapper(hc), max);
+  }
+  
+  /** Expert: Collects matching documents in a range.  Hook for optimization.
+   * Note that {@link #next()} must be called once before this method is called
+   * for the first time.
+   * @param collector The collector to which all matching documents are passed.
+   * @param max Do not score documents past this.
+   * @return true if more matching documents may remain.
+   */
+  protected boolean score(Collector collector, int max) throws IOException {
+    collector.setScorer(this);
     while (doc() < max) {
-      hc.collect(doc(), score());
+      collector.collect(doc());
       if (!next())
         return false;
     }
@@ -78,7 +101,8 @@
 
   /** Returns the score of the current document matching the query.
    * Initially invalid, until {@link #next()} or {@link #skipTo(int)}
-   * is called the first time.
+   * is called the first time, or when called from within
+   * {@link Collector#collect}.
    */
   public abstract float score() throws IOException;
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/Searchable.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/Searchable.java?rev=764551&r1=764550&r2=764551&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/Searchable.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/Searchable.java Mon Apr 13 18:33:56 2009
@@ -19,7 +19,7 @@
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.FieldSelector;
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader; // for javadoc
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.CorruptIndexException;
 
@@ -51,10 +51,32 @@
    * @param filter if non-null, used to permit documents to be collected.
    * @param results to receive hits
    * @throws BooleanQuery.TooManyClauses
+   * @deprecated use {@link #search(Weight, Filter, Collector)} instead.
    */
   void search(Weight weight, Filter filter, HitCollector results)
   throws IOException;
 
+  /**
+   * Lower-level search API.
+   * 
+   * <p>
+   * {@link Collector#collect(int)} is called for every document. <br>
+   * Collector-based access to remote indexes is discouraged.
+   * 
+   * <p>
+   * Applications should only use this if they need <i>all</i> of the matching
+   * documents. The high-level search API ({@link Searcher#search(Query)}) is
+   * usually more efficient, as it skips non-high-scoring hits.
+   * 
+   * @param weight
+   *          to match documents
+   * @param filter
+   *          if non-null, used to permit documents to be collected.
+   * @param collector
+   *          to receive hits
+   * @throws BooleanQuery.TooManyClauses
+   */
+  void search(Weight weight, Filter filter, Collector collector) throws IOException;
 
   /** Frees resources associated with this Searcher.
    * Be careful not to call this method while you are still using objects
@@ -140,6 +162,7 @@
    */
   Explanation explain(Weight weight, int doc) throws IOException;
 
+  // TODO: change the javadoc in 3.0 to remove the last NOTE section.
   /** Expert: Low-level search implementation with arbitrary sorting.  Finds
    * the top <code>n</code> hits for <code>query</code>, applying
    * <code>filter</code> if non-null, and sorting the hits by the criteria in
@@ -147,6 +170,13 @@
    *
    * <p>Applications should usually call {@link
    * Searcher#search(Query,Filter,Sort)} instead.
+   * 
+   * <b>NOTE:</b> currently, this method tracks document scores and sets them in
+   * the returned {@link FieldDoc}, however in 3.0 it will move to not track
+   * document scores. If document scores tracking is still needed, you can use
+   * {@link #search(Weight, Filter, Collector)} and pass in a
+   * {@link TopFieldCollector} instance.
+   * 
    * @throws BooleanQuery.TooManyClauses
    */
   TopFieldDocs search(Weight weight, Filter filter, int n, Sort sort)

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/Searcher.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/Searcher.java?rev=764551&r1=764550&r2=764551&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/Searcher.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/Searcher.java Mon Apr 13 18:33:56 2009
@@ -76,9 +76,13 @@
    * the top <code>n</code> hits for <code>query</code>, applying
    * <code>filter</code> if non-null, and sorting the hits by the criteria in
    * <code>sort</code>.
+   * 
+   * <b>NOTE:</b> currently, this method tracks document scores and sets them in
+   * the returned {@link FieldDoc}, however in 3.0 it will move to not track
+   * document scores. If document scores tracking is still needed, you can use
+   * {@link #search(Weight, Filter, Collector)} and pass in a
+   * {@link TopFieldCollector} instance.
    *
-   * <p>Applications should usually call {@link
-   * Searcher#search(Query,Filter,Sort)} instead.
    * @throws BooleanQuery.TooManyClauses
    */
   public TopFieldDocs search(Query query, Filter filter, int n,
@@ -99,6 +103,7 @@
    * In other words, the score will not necessarily be a float whose value is
    * between 0 and 1.
    * @throws BooleanQuery.TooManyClauses
+   * @deprecated use {@link #search(Query, Collector)} instead.
    */
   public void search(Query query, HitCollector results)
     throws IOException {
@@ -106,6 +111,24 @@
   }
 
   /** Lower-level search API.
+  *
+  * <p>{@link Collector#collect(int)} is called for every matching document.
+  *
+  * <p>Applications should only use this if they need <i>all</i> of the
+  * matching documents.  The high-level search API ({@link
+  * Searcher#search(Query)}) is usually more efficient, as it skips
+  * non-high-scoring hits.
+  * <p>Note: The <code>score</code> passed to this method is a raw score.
+  * In other words, the score will not necessarily be a float whose value is
+  * between 0 and 1.
+  * @throws BooleanQuery.TooManyClauses
+  */
+ public void search(Query query, Collector results)
+   throws IOException {
+   search(query, (Filter)null, results);
+ }
+
+  /** Lower-level search API.
    *
    * <p>{@link HitCollector#collect(int,float)} is called for every matching
    * document.
@@ -120,11 +143,33 @@
    * @param filter if non-null, used to permit documents to be collected.
    * @param results to receive hits
    * @throws BooleanQuery.TooManyClauses
+   * @deprecated use {@link #search(Query, Filter, Collector)} instead.
    */
   public void search(Query query, Filter filter, HitCollector results)
     throws IOException {
     search(createWeight(query), filter, results);
   }
+  
+  /** Lower-level search API.
+   *
+   * <p>{@link Collector#collect(int)} is called for every matching
+   * document.
+   * <br>Collector-based access to remote indexes is discouraged.
+   *
+   * <p>Applications should only use this if they need <i>all</i> of the
+   * matching documents.  The high-level search API ({@link
+   * Searcher#search(Query, Filter, int)}) is usually more efficient, as it skips
+   * non-high-scoring hits.
+   *
+   * @param query to match documents
+   * @param filter if non-null, used to permit documents to be collected.
+   * @param results to receive hits
+   * @throws BooleanQuery.TooManyClauses
+   */
+  public void search(Query query, Filter filter, Collector results)
+  throws IOException {
+    search(createWeight(query), filter, results);
+  }
 
   /** Finds the top <code>n</code>
    * hits for <code>query</code>, applying <code>filter</code> if non-null.
@@ -197,7 +242,11 @@
   /* The following abstract methods were added as a workaround for GCJ bug #15411.
    * http://gcc.gnu.org/bugzilla/show_bug.cgi?id=15411
    */
+  /**
+   * @deprecated use {@link #search(Weight, Filter, Collector)} instead.
+   */
   abstract public void search(Weight weight, Filter filter, HitCollector results) throws IOException;
+  abstract public void search(Weight weight, Filter filter, Collector results) throws IOException;
   abstract public void close() throws IOException;
   abstract public int docFreq(Term term) throws IOException;
   abstract public int maxDoc() throws IOException;

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/SortField.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/SortField.java?rev=764551&r1=764550&r2=764551&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/SortField.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/SortField.java Mon Apr 13 18:33:56 2009
@@ -21,8 +21,6 @@
 import java.io.Serializable;
 import java.util.Locale;
 
-import org.apache.lucene.index.IndexReader;
-
 /**
  * Stores information about how to sort documents by terms in an individual
  * field.  Fields must be indexed in order to sort by them.
@@ -434,8 +432,6 @@
 
 
   /** Returns the {@link FieldComparator} to use for sorting.
-   * @param subReaders array of {@link IndexReader} search
-   *   will step through
    * @param numHits number of top hits the queue will store
    * @param sortPos position of this SortField within {@link
    *   Sort}.  The comparator is primary if sortPos==0,
@@ -444,7 +440,7 @@
    * @param reversed True if the SortField is reversed
    * @return {@link FieldComparator} to use when sorting
    */
-  protected FieldComparator getComparator(final IndexReader[] subReaders, final int numHits, final int sortPos, final boolean reversed) throws IOException {
+  protected FieldComparator getComparator(final int numHits, final int sortPos, final boolean reversed) throws IOException {
 
     if (locale != null) {
       // TODO: it'd be nice to allow FieldCache.getStringIndex
@@ -480,7 +476,7 @@
 
     case SortField.CUSTOM:
       assert factory == null && comparatorSource != null;
-      return comparatorSource.newComparator(field, subReaders, numHits, sortPos, reversed);
+      return comparatorSource.newComparator(field, numHits, sortPos, reversed);
 
     case SortField.STRING:
       return new FieldComparator.StringOrdValComparator(numHits, field, sortPos, reversed);

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/TermScorer.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/TermScorer.java?rev=764551&r1=764550&r2=764551&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/TermScorer.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/TermScorer.java Mon Apr 13 18:33:56 2009
@@ -24,6 +24,9 @@
 /** Expert: A <code>Scorer</code> for documents matching a <code>Term</code>.
  */
 final class TermScorer extends Scorer {
+  
+  private static final float[] SIM_NORM_DECODER = Similarity.getNormDecoder();
+  
   private Weight weight;
   private TermDocs termDocs;
   private byte[] norms;
@@ -56,25 +59,26 @@
       scoreCache[i] = getSimilarity().tf(i) * weightValue;
   }
 
+  /** @deprecated use {@link #score(Collector)} instead. */
   public void score(HitCollector hc) throws IOException {
+    score(new HitCollectorWrapper(hc));
+  }
+
+  public void score(Collector c) throws IOException {
     next();
-    score(hc, Integer.MAX_VALUE);
+    score(c, Integer.MAX_VALUE);
   }
 
+  /** @deprecated use {@link #score(Collector, int)} instead. */
   protected boolean score(HitCollector c, int end) throws IOException {
-    Similarity similarity = getSimilarity();      // cache sim in local
-    float[] normDecoder = Similarity.getNormDecoder();
+    return score(new HitCollectorWrapper(c), end);
+  }
+  
+  protected boolean score(Collector c, int end) throws IOException {
+    c.setScorer(this);
     while (doc < end) {                           // for docs in window
-      int f = freqs[pointer];
-      float score =                               // compute tf(f)*weight
-        f < SCORE_CACHE_SIZE                      // check cache
-         ? scoreCache[f]                          // cache hit
-         : similarity.tf(f)*weightValue;          // cache miss
-
-      score *= normDecoder[norms[doc] & 0xFF];    // normalize for field
-
-      c.collect(doc, score);                      // collect score
-
+      c.collect(doc);                      // collect score
+        
       if (++pointer >= pointerMax) {
         pointerMax = termDocs.read(docs, freqs);  // refill buffers
         if (pointerMax != 0) {
@@ -123,7 +127,7 @@
       ? scoreCache[f]                             // cache hit
       : getSimilarity().tf(f)*weightValue;        // cache miss
 
-    return raw * Similarity.decodeNorm(norms[doc]); // normalize for field
+    return raw * SIM_NORM_DECODER[norms[doc] & 0xFF]; // normalize for field
   }
 
   /** Skips to the first match beyond the current whose document number is

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/TimeLimitedCollector.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/TimeLimitedCollector.java?rev=764551&r1=764550&r2=764551&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/TimeLimitedCollector.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/TimeLimitedCollector.java Mon Apr 13 18:33:56 2009
@@ -1,7 +1,5 @@
 package org.apache.lucene.search;
 
-import org.apache.lucene.index.IndexReader;
-
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -17,18 +15,20 @@
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  * See the License for the specific language governing permissions and
  * limitations under the License.
- */
-
-import java.io.IOException;
+*/
 
 /**
- * <p>The TimeLimitedCollector is used to timeout search requests that
- * take longer than the maximum allowed search time limit.  After this
- * time is exceeded, the search thread is stopped by throwing a
- * TimeExceeded Exception.</p>
+ * <p>
+ * The TimeLimitedCollector is used to timeout search requests that take longer
+ * than the maximum allowed search time limit. After this time is exceeded, the
+ * search thread is stopped by throwing a TimeExceeded Exception.
+ * </p>
  * 
+ * @deprecated this class will be removed in 3.0. Use
+ *             {@link TimeLimitingCollector} instead, which extends the new
+ *             {@link Collector}.
  */
-public class TimeLimitedCollector extends MultiReaderHitCollector {
+public class TimeLimitedCollector extends HitCollector {
   
   /** 
    * Default timer resolution.
@@ -136,19 +136,15 @@
 
   private final long t0;
   private final long timeout;
-  private final MultiReaderHitCollector hc;
+  private final HitCollector hc;
 
   /**
    * Create a TimeLimitedCollector wrapper over another HitCollector with a specified timeout.
    * @param hc the wrapped HitCollector
    * @param timeAllowed max time allowed for collecting hits after which {@link TimeExceededException} is thrown
    */
-  public TimeLimitedCollector( final HitCollector hc, final long timeAllowed ) {
-    if (hc instanceof MultiReaderHitCollector) {
-      this.hc = (MultiReaderHitCollector) hc;
-    } else {
-      this.hc = new IndexSearcher.MultiReaderCollectorWrapper(hc);
-    }
+  public TimeLimitedCollector(final HitCollector hc, final long timeAllowed) {
+    this.hc = hc;
     t0 = TIMER_THREAD.getMilliseconds();
     this.timeout = t0 + timeAllowed;
   }
@@ -219,7 +215,4 @@
     this.greedy = greedy;
   }
   
-  public void setNextReader(IndexReader reader, int base) throws IOException {
-    hc.setNextReader(reader, base);
-  }
 }

Added: lucene/java/trunk/src/java/org/apache/lucene/search/TimeLimitingCollector.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/TimeLimitingCollector.java?rev=764551&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/TimeLimitingCollector.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/TimeLimitingCollector.java Mon Apr 13 18:33:56 2009
@@ -0,0 +1,219 @@
+package org.apache.lucene.search;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.index.IndexReader;
+
+/**
+ * The {@link TimeLimitingCollector} is used to timeout search requests that
+ * take longer than the maximum allowed search time limit. After this time is
+ * exceeded, the search thread is stopped by throwing a
+ * {@link TimeExceededException}.
+ */
+public class TimeLimitingCollector extends Collector {
+
+  /** 
+   * Default timer resolution.
+   * @see #setResolution(long) 
+   */
+  public static final int DEFAULT_RESOLUTION = 20;
+
+  /**
+   * Default for {@link #isGreedy()}.
+   * @see #isGreedy()
+   */
+  public boolean DEFAULT_GREEDY = false; 
+
+  private static long resolution = DEFAULT_RESOLUTION;
+  
+  private boolean greedy = DEFAULT_GREEDY ;
+
+  private static final class TimerThread extends Thread  {
+
+    // NOTE: we can avoid explicit synchronization here for several reasons:
+    // * updates to volatile long variables are atomic
+    // * only single thread modifies this value
+    // * use of volatile keyword ensures that it does not reside in
+    //   a register, but in main memory (so that changes are visible to
+    //   other threads).
+    // * visibility of changes does not need to be instantanous, we can
+    //   afford losing a tick or two.
+    //
+    // See section 17 of the Java Language Specification for details.
+    private volatile long time = 0;
+
+    /**
+     * TimerThread provides a pseudo-clock service to all searching
+     * threads, so that they can count elapsed time with less overhead
+     * than repeatedly calling System.currentTimeMillis.  A single
+     * thread should be created to be used for all searches.
+     */
+    private TimerThread() {
+      super("TimeLimitedCollector timer thread");
+      this.setDaemon( true );
+    }
+
+    public void run() {
+      while (true) {
+        // TODO: Use System.nanoTime() when Lucene moves to Java SE 5.
+        time += resolution;
+        try {
+          Thread.sleep( resolution );
+        } catch (InterruptedException ie) {
+          Thread.currentThread().interrupt();
+          throw new RuntimeException(ie);
+        }
+      }
+    }
+
+    /**
+     * Get the timer value in milliseconds.
+     */
+    public long getMilliseconds() {
+      return time;
+    }
+  }
+
+  /** Thrown when elapsed search time exceeds allowed search time. */
+  public static class TimeExceededException extends RuntimeException {
+    private long timeAllowed;
+    private long timeElapsed;
+    private int lastDocCollected;
+    private TimeExceededException(long timeAllowed, long timeElapsed, int lastDocCollected) {
+      super("Elapsed time: " + timeElapsed + "Exceeded allowed search time: " + timeAllowed + " ms.");
+      this.timeAllowed = timeAllowed;
+      this.timeElapsed = timeElapsed;
+      this.lastDocCollected = lastDocCollected;
+    }
+    /** Returns allowed time (milliseconds). */
+    public long getTimeAllowed() {
+      return timeAllowed;
+    }
+    /** Returns elapsed time (milliseconds). */
+    public long getTimeElapsed() {
+      return timeElapsed;
+    }
+    /** Returns last doc that was collected when the search time exceeded. */
+    public int getLastDocCollected() {
+      return lastDocCollected;
+    }
+  }
+
+  // Declare and initialize a single static timer thread to be used by
+  // all TimeLimitedCollector instances.  The JVM assures that
+  // this only happens once.
+  private final static TimerThread TIMER_THREAD = new TimerThread();
+  
+  static  {
+    TIMER_THREAD.start();
+  }
+
+  private final long t0;
+  private final long timeout;
+  private final Collector collector;
+
+  /**
+   * Create a TimeLimitedCollector wrapper over another {@link Collector} with a specified timeout.
+   * @param collector the wrapped {@link Collector}
+   * @param timeAllowed max time allowed for collecting hits after which {@link TimeExceededException} is thrown
+   */
+  public TimeLimitingCollector(final Collector collector, final long timeAllowed ) {
+    this.collector = collector;
+    t0 = TIMER_THREAD.getMilliseconds();
+    this.timeout = t0 + timeAllowed;
+  }
+
+  /** 
+   * Return the timer resolution.
+   * @see #setResolution(long)
+   */
+  public static long getResolution() {
+    return resolution;
+  }
+
+  /**
+   * Set the timer resolution.
+   * The default timer resolution is 20 milliseconds. 
+   * This means that a search required to take no longer than 
+   * 800 milliseconds may be stopped after 780 to 820 milliseconds.
+   * <br>Note that: 
+   * <ul>
+   * <li>Finer (smaller) resolution is more accurate but less efficient.</li>
+   * <li>Setting resolution to less than 5 milliseconds will be silently modified to 5 milliseconds.</li>
+   * <li>Setting resolution smaller than current resolution might take effect only after current 
+   * resolution. (Assume current resolution of 20 milliseconds is modified to 5 milliseconds, 
+   * then it can take up to 20 milliseconds for the change to have effect.</li>
+   * </ul>      
+   */
+  public static void setResolution(long newResolution) {
+    resolution = Math.max(newResolution,5); // 5 milliseconds is about the minimum reasonable time for a Object.wait(long) call.
+  }
+
+  /**
+   * Checks if this time limited collector is greedy in collecting the last hit.
+   * A non greedy collector, upon a timeout, would throw a {@link TimeExceededException} 
+   * without allowing the wrapped collector to collect current doc. A greedy one would 
+   * first allow the wrapped hit collector to collect current doc and only then 
+   * throw a {@link TimeExceededException}.
+   * @see #setGreedy(boolean)
+   */
+  public boolean isGreedy() {
+    return greedy;
+  }
+
+  /**
+   * Sets whether this time limited collector is greedy.
+   * @param greedy true to make this time limited greedy
+   * @see #isGreedy()
+   */
+  public void setGreedy(boolean greedy) {
+    this.greedy = greedy;
+  }
+  
+  /**
+   * Calls {@link Collector#collect(int)} on the decorated {@link Collector}
+   * unless the allowed time has passed, in which case it throws an exception.
+   * 
+   * @throws TimeExceededException
+   *           if the time allowed has exceeded.
+   */
+  public void collect(final int doc) throws IOException {
+    long time = TIMER_THREAD.getMilliseconds();
+    if (timeout < time) {
+      if (greedy) {
+        //System.out.println(this+"  greedy: before failing, collecting doc: "+doc+"  "+(time-t0));
+        collector.collect(doc);
+      }
+      //System.out.println(this+"  failing on:  "+doc+"  "+(time-t0));
+      throw new TimeExceededException( timeout-t0, time-t0, doc );
+    }
+    //System.out.println(this+"  collecting: "+doc+"  "+(time-t0));
+    collector.collect(doc);
+  }
+  
+  public void setNextReader(IndexReader reader, int base) throws IOException {
+    collector.setNextReader(reader, base);
+  }
+  
+  public void setScorer(Scorer scorer) throws IOException {
+    collector.setScorer(scorer);
+  }
+
+}

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/TopDocs.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/TopDocs.java?rev=764551&r1=764550&r2=764551&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/TopDocs.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/TopDocs.java Mon Apr 13 18:33:56 2009
@@ -29,7 +29,10 @@
   /** Expert: Stores the maximum score value encountered, needed for normalizing. */
   private float maxScore;
   
-  /** Expert: Returns the maximum score value encountered. */
+  /**
+   * Expert: Returns the maximum score value encountered. Note that in case
+   * scores are not tracked, this returns {@link Float#NaN}.
+   */
   public float getMaxScore() {
       return maxScore;
   }
@@ -38,7 +41,12 @@
   public void setMaxScore(float maxScore) {
       this.maxScore=maxScore;
   }
-  
+
+  /** Expert: Constructs a TopDocs with a default maxScore=Float.NaN. */
+  TopDocs(int totalHits, ScoreDoc[] scoreDocs) {
+    this(totalHits, scoreDocs, Float.NaN);
+  }
+
   /** Expert: Constructs a TopDocs.*/
   public TopDocs(int totalHits, ScoreDoc[] scoreDocs, float maxScore) {
     this.totalHits = totalHits;

Added: 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=764551&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/TopDocsCollector.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/TopDocsCollector.java Mon Apr 13 18:33:56 2009
@@ -0,0 +1,138 @@
+package org.apache.lucene.search;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.util.PriorityQueue;
+
+/**
+ * A base class for all collectors that return a {@link TopDocs} output. This
+ * collector allows easy extension by providing a single constructor which
+ * accepts a {@link PriorityQueue} as well as protected members for that
+ * priority queue and a counter of the number of total hits.<br>
+ * Extending classes can override {@link #topDocs(int, int)} and
+ * {@link #getTotalHits()} in order to provide their own implementation.
+ */
+public abstract class TopDocsCollector extends Collector {
+
+  // This is used in case topDocs() is called with illegal parameters, or there
+  // simply aren't (enough) results.
+  protected static final TopDocs EMPTY_TOPDOCS = new TopDocs(0, new ScoreDoc[0], Float.NaN);
+  
+  /**
+   * The priority queue which holds the top documents. Note that different
+   * implementations of PriorityQueue give different meaning to 'top documents'.
+   * HitQueue for example aggregates the top scoring documents, while other PQ
+   * implementations may hold documents sorted by other criteria.
+   */
+  protected PriorityQueue pq;
+
+  /** The total number of documents that the collector encountered. */
+  protected int totalHits;
+  
+  protected TopDocsCollector(PriorityQueue pq) {
+    this.pq = pq;
+  }
+  
+  /**
+   * Populates the results array with the ScoreDoc instaces. This can be
+   * overridden in case a different ScoreDoc type should be returned.
+   */
+  protected void populateResults(ScoreDoc[] results, int howMany) {
+    for (int i = howMany - 1; i >= 0; i--) { 
+      results[i] = (ScoreDoc) pq.pop();
+    }
+  }
+
+  /**
+   * Returns a {@link TopDocs} instance containing the given results. If
+   * <code>results</code> is null it means there are no results to return,
+   * either because there were 0 calls to collect() or because the arguments to
+   * topDocs were invalid.
+   */
+  protected TopDocs newTopDocs(ScoreDoc[] results, int start) {
+    return results == null ? EMPTY_TOPDOCS : new TopDocs(totalHits, results);
+  }
+  
+  /** The total number of documents that matched this query. */
+  public int getTotalHits() {
+    return totalHits;
+  }
+  
+  /** Returns the top docs that were collected by this collector. */
+  public final TopDocs topDocs() {
+    return topDocs(0, pq.size());
+  }
+
+  /**
+   * Returns the documents in the rage [start .. pq.size()) that were collected
+   * by this collector. Note that if start >= pq.size(), an empty TopDocs is
+   * returned.<br>
+   * This method is convenient to call if the application allways asks for the
+   * last results, starting from the last 'page'.<br>
+   * <b>NOTE:</b> you cannot call this method more than once for each search
+   * execution. If you need to call it more than once, passing each time a
+   * different <code>start</code>, you should call {@link #topDocs()} and work
+   * with the returned {@link TopDocs} object, which will contain all the
+   * results this search execution collected.
+   */
+  public final TopDocs topDocs(int start) {
+    return topDocs(start, pq.size());
+  }
+
+  /**
+   * Returns the documents in the rage [start .. start+howMany) that were
+   * collected by this collector. Note that if start >= pq.size(), an empty
+   * TopDocs is returned, and if pq.size() - start &lt; howMany, then only the
+   * available documents in [start .. pq.size()) are returned.<br>
+   * This method is useful to call in case pagination of search results is
+   * allowed by the search application, as well as it attempts to optimize the
+   * memory used by allocating only as much as requested by howMany.<br>
+   * <b>NOTE:</b> you cannot call this method more than once for each search
+   * execution. If you need to call it more than once, passing each time a
+   * different range, you should call {@link #topDocs()} and work with the
+   * returned {@link TopDocs} object, which will contain all the results this
+   * search execution collected.
+   */
+  public TopDocs topDocs(int start, int howMany) {
+    
+    int pqsize = 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) {
+      return newTopDocs(null, start);
+    }
+
+    // We know that start < pqsize, so just fix howMany. 
+    howMany = Math.min(pqsize - start, howMany);
+    ScoreDoc[] results = new ScoreDoc[howMany];
+
+    // pq's pop() returns the 'least' element in the queue, therefore need
+    // to discard the first ones, until we reach the requested range.
+    // 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(); }
+    
+    // Get the requested results from pq.
+    populateResults(results, howMany);
+    
+    return newTopDocs(results, start);
+  }
+
+}

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=764551&r1=764550&r2=764551&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 Mon Apr 13 18:33:56 2009
@@ -21,201 +21,569 @@
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.search.FieldValueHitQueue.Entry;
+import org.apache.lucene.util.PriorityQueue;
 
 /**
- * A {@link HitCollector} that sorts by {@link SortField} using 
+ * A {@link Collector} that sorts by {@link SortField} using
  * {@link FieldComparator}s.
- *
- * <b>NOTE:</b> This API is experimental and might change in
- * incompatible ways in the next release.
+ * 
+ * <p><b>NOTE:</b> This API is experimental and might change in
+ * incompatible ways in the next release.</p>
  */
-public final class TopFieldCollector extends MultiReaderHitCollector {
-
-  private final FieldValueHitQueue queue;
-
-  private final FieldComparator[] comparators;
-  private FieldComparator comparator1;
-  private final int numComparators;
-  private int[] reverseMul;
-  private int reverseMul1 = 0;
-
-  private final int numHits;
-  private int totalHits;
-  private FieldValueHitQueue.Entry bottom = null;
-
-  /** Stores the maximum score value encountered, needed for normalizing. */
-  private float maxScore = Float.NEGATIVE_INFINITY;
+public abstract class TopFieldCollector extends TopDocsCollector {
+  
+  // 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 {
+
+    final FieldComparator comparator;
+    final int reverseMul;
+    
+    public OneComparatorNonScoringCollector(FieldValueHitQueue queue,
+        int numHits, boolean fillFields) throws IOException {
+      super(queue, numHits, fillFields);
+      comparator = queue.getComparators()[0];
+      reverseMul = queue.getReverseMul()[0];
+    }
+    
+    private 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();
+    }
 
-  private boolean queueFull;
+    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);
+        }
+      }
+    }
+    
+    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 {
+      comparator.setScorer(scorer);
+    }
+    
+  }
 
-  private boolean fillFields;
+  /*
+   * Implements a TopFieldCollector over one SortField criteria, while tracking
+   * document scores but no maxScore.
+   */
+  private static class OneComparatorScoringNoMaxScoreCollector extends
+      OneComparatorNonScoringCollector {
+
+    private Scorer scorer;
+
+    public OneComparatorScoringNoMaxScoreCollector(FieldValueHitQueue queue,
+        int numHits, boolean fillFields) throws IOException {
+      super(queue, numHits, fillFields);
+    }
+    
+    private final void updateBottom(int doc, float score) {
+      bottom.docID = docBase + doc;
+      bottom.score = score;
+      pq.adjustTop();
+      bottom = (FieldValueHitQueue.Entry) pq.top();
+    }
 
-  public TopFieldCollector(Sort sort, int numHits,  IndexReader[] subReaders, boolean fillFields)
-      throws IOException {
+    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();
 
-    if (sort.fields.length == 0) {
-      throw new IllegalArgumentException("Sort must contain at least one field");
+        // 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);
+        }
+      }
     }
-
-    queue = new FieldValueHitQueue(sort.fields, numHits, subReaders);
-    comparators = queue.getComparators(); 
-    reverseMul = queue.getReverseMul(); 
-    numComparators = comparators.length;
-
-    if (numComparators == 1) {
-      comparator1 = comparators[0];
-      reverseMul1 = reverseMul[0];
-    } else {
-      comparator1 = null;
-      reverseMul1 = 0;
+    
+    public void setNextReader(IndexReader reader, int docBase) throws IOException {
+      final int numSlotsFull = queueFull ? numHits : totalHits;
+      this.docBase = docBase;
+      comparator.setNextReader(reader, docBase, numSlotsFull);
     }
-    this.numHits = numHits;
-    this.fillFields = fillFields;
+    
+    public void setScorer(Scorer scorer) throws IOException {
+      this.scorer = scorer;
+      comparator.setScorer(scorer);
+    }
+    
   }
 
-  int currentDocBase;
-
-  // javadoc inherited
-  public void setNextReader(IndexReader reader, int docBase) throws IOException {
-    final int numSlotsFull;
-    if (queueFull)
-      numSlotsFull = numHits;
-    else
-      numSlotsFull = totalHits;
+  /*
+   * Implements a TopFieldCollector over one SortField criteria, with tracking
+   * document scores and maxScore.
+   */
+  private final static class OneComparatorScoringMaxScoreCollector extends
+      OneComparatorNonScoringCollector {
+
+    private Scorer scorer;
+    
+    public OneComparatorScoringMaxScoreCollector(FieldValueHitQueue queue,
+        int numHits, boolean fillFields) throws IOException {
+      super(queue, numHits, fillFields);
+      // Must set maxScore to NEG_INF, or otherwise Math.max always returns NaN.
+      maxScore = Float.NEGATIVE_INFINITY;
+    }
+    
+    private final void updateBottom(int doc, float score) {
+      bottom.docID = docBase + doc;
+      bottom.score = score;
+      pq.adjustTop();
+      bottom = (FieldValueHitQueue.Entry) pq.top();
+    }
 
-    currentDocBase = docBase;
+    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);
+        }
+      }
 
-    for (int i = 0; i < numComparators; i++) {
-      comparators[i].setNextReader(reader, docBase, numSlotsFull);
+    }
+    
+    public void setScorer(Scorer scorer) throws IOException {
+      this.scorer = scorer;
+      super.setScorer(scorer);
     }
   }
 
-  private final void updateBottom(int doc, float score) {
-    bottom.docID = currentDocBase + doc;
-    bottom.score = score;
-    queue.adjustTop();
-    bottom = (FieldValueHitQueue.Entry) queue.top();
-  }
+  /*
+   * Implements a TopFieldCollector over multiple SortField criteria, without
+   * tracking document scores and maxScore.
+   */
+  private static class MultiComparatorNonScoringCollector extends TopFieldCollector {
+    
+    final FieldComparator[] comparators;
+    final int[] reverseMul;
+    
+    public MultiComparatorNonScoringCollector(FieldValueHitQueue queue,
+        int numHits, boolean fillFields) throws IOException {
+      super(queue, numHits, fillFields);
+      comparators = queue.getComparators();
+      reverseMul = queue.getReverseMul();
+    }
+    
+    private 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();
+    }
 
-  private final void add(int slot, int doc, float score) {
-    queue.put(new FieldValueHitQueue.Entry(slot, currentDocBase+doc, score));
-    bottom = (FieldValueHitQueue.Entry) queue.top();
-    queueFull = totalHits == numHits;
-  }     
+    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;
+          }
+        }
 
-  // javadoc inherited
-  public void collect(int doc, float score) {
-    if (score > 0.0f) {
+        // This hit is competitive - replace bottom element in queue & adjustTop
+        for (int i = 0; i < comparators.length; i++) {
+          comparators[i].copy(bottom.slot, doc);
+        }
 
-      maxScore = Math.max(maxScore, score);
-      totalHits++;
+        updateBottom(doc);
 
-      // 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
+        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);
+          }
+        }
+      }
+    }
 
-      if (queueFull) {
+    public void setNextReader(IndexReader reader, int docBase) throws IOException {
+      final int numSlotsFull = queueFull ? numHits : totalHits;
+      this.docBase = docBase;
+      for (int i = 0; i < comparators.length; i++) {
+        comparators[i].setNextReader(reader, docBase, numSlotsFull);
+      }
+    }
 
-        if (numComparators == 1) {
-          // Common case
+    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, with
+   * tracking document scores and maxScore.
+   */
+  private final static class MultiComparatorScoringMaxScoreCollector extends MultiComparatorNonScoringCollector {
+    
+    private Scorer scorer;
+    
+    public MultiComparatorScoringMaxScoreCollector(FieldValueHitQueue queue,
+        int numHits, boolean fillFields) throws IOException {
+      super(queue, numHits, fillFields);
+      // Must set maxScore to NEG_INF, or otherwise Math.max always returns NaN.
+      maxScore = Float.NEGATIVE_INFINITY;
+    }
+    
+    private final void updateBottom(int doc, float score) {
+      bottom.docID = docBase + doc;
+      bottom.score = score;
+      pq.adjustTop();
+      bottom = (FieldValueHitQueue.Entry) pq.top();
+    }
 
-          // Fastmatch: return if this hit is not competitive
-          final int cmp = reverseMul1 * comparator1.compareBottom(doc, score);
-          if (cmp < 0) {
-            // Definitely not competitive
-            return;
-          } else if (cmp == 0 && doc + currentDocBase > bottom.docID) {
+    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.docID) {
+              // Definitely not competitive
+              return;
+            }
+            break;
           }
+        }
 
-          // This hit is competitive -- replace bottom
-          // element in queue & adjustTop
-          comparator1.copy(bottom.slot, doc, score);
+        // 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);
+        updateBottom(doc, score);
 
-          comparator1.setBottom(bottom.slot);
+        for (int i = 0; i < comparators.length; i++) {
+          comparators[i].setBottom(bottom.slot);
+        }
+      } else {
+        // Startup transient: queue hasn't gathered numHits yet
+        final int slot = totalHits - 1;
+        // Copy hit into queue
+        for (int i = 0; i < comparators.length; i++) {
+          comparators[i].copy(slot, doc);
+        }
+        add(slot, doc, score);
+        if (queueFull) {
+          for (int i = 0; i < comparators.length; i++) {
+            comparators[i].setBottom(bottom.slot);
+          }
+        }
+      }
+    }
 
-        } else {
+    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 Scorer scorer;
+    
+    public MultiComparatorScoringNoMaxScoreCollector(FieldValueHitQueue queue,
+        int numHits, boolean fillFields) throws IOException {
+      super(queue, numHits, fillFields);
+    }
+    
+    private final void updateBottom(int doc, float score) {
+      bottom.docID = docBase + doc;
+      bottom.score = score;
+      pq.adjustTop();
+      bottom = (FieldValueHitQueue.Entry) pq.top();
+    }
 
-          // Fastmatch: return if this hit is not competitive
-          for(int i=0;;i++) {
-            final int c = reverseMul[i] * comparators[i].compareBottom(doc, score);
-            if (c < 0) {
+    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;
-            } else if (c > 0) {
-              // Definitely competitive
-              break;
-            } else if (i == numComparators-1) {
-              // This is the equals case.
-              if (doc + currentDocBase > bottom.docID) {
-                // Definitely not competitive
-                return;
-              } else {
-                break;
-              }
             }
+            break;
           }
+        }
 
-          // This hit is competitive -- replace bottom
-          // element in queue & adjustTop
-          for (int i = 0; i < numComparators; i++) {
-            comparators[i].copy(bottom.slot, doc, score);
-          }
+        // 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);
+        // Compute score only if it is competitive.
+        final float score = scorer.score();
+        updateBottom(doc, score);
 
-          for(int i=0;i<numComparators;i++) {
-            comparators[i].setBottom(bottom.slot);
-          }
+        for (int i = 0; i < comparators.length; i++) {
+          comparators[i].setBottom(bottom.slot);
         }
       } else {
-        // Startup transient: queue hasn't gathered numHits
-        // yet
-
-        final int slot = totalHits-1;
+        // Startup transient: queue hasn't gathered numHits yet
+        final int slot = totalHits - 1;
         // Copy hit into queue
-        if (numComparators == 1) {
-          // Common case
-          comparator1.copy(slot, doc, score);
-          add(slot, doc, score);
-          if (queueFull) {
-            comparator1.setBottom(bottom.slot);
-          }
-
-        } else {
-          for (int i = 0; i < numComparators; i++) {
-            comparators[i].copy(slot, doc, score);
-          }
-          add(slot, doc, score);
-          if (queueFull) {
-            for(int i=0;i<numComparators;i++) {
-              comparators[i].setBottom(bottom.slot);
-            }
+        for (int i = 0; i < comparators.length; i++) {
+          comparators[i].copy(slot, doc);
+        }
+
+        // Compute score only if it 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);
+    }
+  }
+
+  private static final ScoreDoc[] EMPTY_SCOREDOCS = new ScoreDoc[0];
+  
+  private final boolean fillFields;
+
+  /*
+   * Stores the maximum score value encountered, needed for normalizing. If
+   * document scores are not tracked, this value is initialized to NaN.
+   */
+  float maxScore = Float.NaN;
+
+  final int numHits;
+  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
+  // visibility, then anyone will be able to extend the class, which is not what
+  // we want.
+  private TopFieldCollector(PriorityQueue pq, int numHits, boolean fillFields) {
+    super(pq);
+    this.numHits = numHits;
+    this.fillFields = fillFields;
   }
 
-  // javadoc inherited
-  public TopDocs topDocs() {
-    ScoreDoc[] scoreDocs = new ScoreDoc[queue.size()];
+  /**
+   * Creates a new {@link TopFieldCollector} from the given arguments.
+   * 
+   * @param sort
+   *          the sort criteria (SortFields).
+   * @param numHits
+   *          the number of results to collect.
+   * @param fillFields
+   *          specifies whether the actual field values should be returned on
+   *          the results (FieldDoc).
+   * @param trackDocScores
+   *          specifies whether document scores should be tracked and set on the
+   *          results. Note that if set to false, then the results' scores will
+   *          be set to Float.NaN. Setting this to true affects performance, as
+   *          it incurs the score computation on each competitive result.
+   *          Therefore if document scores are not required by the application,
+   *          it is recommended to set it to false.
+   * @param trackMaxScore
+   *          specifies whether the query's maxScore should be tracked and set
+   *          on the resulting {@link TopDocs}. Note that if set to false,
+   *          {@link TopDocs#getMaxScore()} returns Float.NaN. Setting this to
+   *          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.
+   * @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)
+      throws IOException {
+    if (sort.fields.length == 0) {
+      throw new IllegalArgumentException("Sort must contain at least one field");
+    }
+    
+    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);
+      } else {
+        return new OneComparatorNonScoringCollector(queue, numHits, fillFields);
+      }
+    }
+
+    // multiple comparators.
+    if (trackMaxScore) {
+      return new MultiComparatorScoringMaxScoreCollector(queue, numHits, fillFields);
+    } else if (trackDocScores) {
+      return new MultiComparatorScoringNoMaxScoreCollector(queue, numHits, fillFields);
+    } else {
+      return new MultiComparatorNonScoringCollector(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();
+    queueFull = totalHits == numHits;
+  }
+
+  /*
+   * Only the following callback methods need to be overridden since
+   * topDocs(int, int) calls them to return the results.
+   */
+
+  protected void populateResults(ScoreDoc[] results, int howMany) {
+    FieldValueHitQueue queue = (FieldValueHitQueue) pq;
     if (fillFields) {
       for (int i = queue.size() - 1; i >= 0; i--) {
-        scoreDocs[i] = queue.fillFields((FieldValueHitQueue.Entry) queue.pop());
+        results[i] = queue.fillFields((FieldValueHitQueue.Entry) queue.pop());
       }
     } else {
-      Entry entry = (FieldValueHitQueue.Entry) queue.pop();
       for (int i = queue.size() - 1; i >= 0; i--) {
-        scoreDocs[i] = new FieldDoc(entry.docID,
-                                    entry.score);
+        Entry entry = (FieldValueHitQueue.Entry) queue.pop();
+        results[i] = new FieldDoc(entry.docID, entry.score);
       }
     }
+  }
+  
+  protected TopDocs newTopDocs(ScoreDoc[] results, int start) {
+    if (results == null) {
+      results = EMPTY_SCOREDOCS;
+      // Set maxScore to NaN, in case this is a maxScore tracking collector.
+      maxScore = Float.NaN;
+    }
 
-    return new TopFieldDocs(totalHits, scoreDocs, queue.getFields(), maxScore);
+    // If this is a maxScoring tracking collector and there were no results, 
+    return new TopFieldDocs(totalHits, results, ((FieldValueHitQueue) pq).getFields(), maxScore);
   }
+  
 }

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/TopFieldDocCollector.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/TopFieldDocCollector.java?rev=764551&r1=764550&r2=764551&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/TopFieldDocCollector.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/TopFieldDocCollector.java Mon Apr 13 18:33:56 2009
@@ -30,8 +30,8 @@
  * documents are collected.
  *
  * @deprecated Please use {@link TopFieldCollector} instead.
- **/
-public class TopFieldDocCollector extends TopScoreDocCollector {
+ */
+public class TopFieldDocCollector extends TopDocCollector {
 
   private FieldDoc reusableFD;
 
@@ -50,7 +50,7 @@
     if (score > 0.0f) {
       totalHits++;
       if (reusableFD == null)
-        reusableFD = new FieldDoc(doc + docBase, score);
+        reusableFD = new FieldDoc(doc, score);
       else {
         // Whereas TopScoreDocCollector can skip this if the
         // score is not competitive, we cannot because the
@@ -58,7 +58,7 @@
         // aren't in general congruent with "higher score
         // wins"
         reusableFD.score = score;
-        reusableFD.doc = doc + docBase;
+        reusableFD.doc = doc;
       }
       reusableFD = (FieldDoc) hq.insertWithOverflow(reusableFD);
     }

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=764551&r1=764550&r2=764551&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 Mon Apr 13 18:33:56 2009
@@ -17,85 +17,74 @@
  * limitations under the License.
  */
 
+import java.io.IOException;
+
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.util.PriorityQueue;
 
-/** A {@link MultiReaderHitCollector} implementation that
- *  collects the top-scoring documents, returning them as a
- *  {@link TopDocs}.  This is used by {@link IndexSearcher}
- *  to implement {@link TopDocs}-based search.
- *
- *  <p>This may be extended, overriding the {@link
- *  MultiReaderHitCollector#collect} method to, e.g.,
- *  conditionally invoke <code>super()</code> in order to
- *  filter which documents are collected, but sure you
- *  either take docBase into account, or also override
- *  {@link MultiReaderHitCollector#setNextReader} method. */
-public class TopScoreDocCollector extends MultiReaderHitCollector {
+/**
+ * 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.
+ */
+public final class TopScoreDocCollector extends TopDocsCollector {
 
   private ScoreDoc reusableSD;
-  
-  /** The total number of hits the collector encountered. */
-  protected int totalHits;
-  
-  /** The priority queue which holds the top-scoring documents. */
-  protected PriorityQueue hq;
-
-  protected int docBase = 0;
+  private int docBase = 0;
+  private Scorer scorer;
     
   /** Construct to collect a given number of hits.
    * @param numHits the maximum number of hits to collect
    */
   public TopScoreDocCollector(int numHits) {
-    this(new HitQueue(numHits));
+    super(new HitQueue(numHits));
   }
 
-  /** Constructor to collect the top-scoring documents by using the given PQ.
-   * @param hq the PQ to use by this instance.
-   */
-  protected TopScoreDocCollector(PriorityQueue hq) {
-    this.hq = hq;
-  }
-
-  // javadoc inherited
-  public void collect(int doc, float score) {
-    if (score > 0.0f) {
-      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) hq.insertWithOverflow(reusableSD);
+  protected TopDocs newTopDocs(ScoreDoc[] results, int start) {
+    if (results == null) {
+      return EMPTY_TOPDOCS;
     }
-  }
-
-  /** The total number of documents that matched this query. */
-  public int getTotalHits() {
-    return totalHits;
-  }
-
-  /** The top-scoring hits. */
-  public TopDocs topDocs() {
-    ScoreDoc[] scoreDocs = new ScoreDoc[hq.size()];
-    for (int i = hq.size()-1; i >= 0; i--) {     // put docs in array
-      scoreDocs[i] = (ScoreDoc) hq.pop();
+    
+    // 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
+    // extracted and use its score as maxScore.
+    float maxScore = Float.NaN;
+    if (start == 0) {
+      maxScore = results[0].score;
+    } else {
+      for (int i = pq.size(); i > 1; i--) { pq.pop(); }
+      maxScore = ((ScoreDoc) pq.pop()).score;
     }
-      
-    float maxScore = (totalHits==0)
-      ? Float.NEGATIVE_INFINITY
-      : scoreDocs[0].score;
     
-    return new TopDocs(totalHits, scoreDocs, maxScore);
+    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;
   }
+  
+  public void setScorer(Scorer scorer) throws IOException {
+    this.scorer = scorer;
+  }
 }

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=764551&r1=764550&r2=764551&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 Mon Apr 13 18:33:56 2009
@@ -23,10 +23,10 @@
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Set;
-import java.util.HashSet;
 
 import junit.framework.TestSuite;
 import junit.textui.TestRunner;
@@ -35,15 +35,16 @@
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
-import org.apache.lucene.document.Fieldable;
 import org.apache.lucene.document.FieldSelector;
+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.IndexSearcher;
-import org.apache.lucene.search.MultiReaderHitCollector;
+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.Query;
 import org.apache.lucene.search.FieldCache;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.Directory;
@@ -1651,7 +1652,7 @@
     Query q = new TermQuery(new Term("id", "a"));
 
     IndexSearcher s = new IndexSearcher(dir);
-    s.search(q, new MultiReaderHitCollector() {
+    s.search(q, new Collector() {
         int lastDocBase = -1;
         public void setNextReader(IndexReader reader, int docBase) {
           if (lastDocBase == -1) {
@@ -1663,13 +1664,14 @@
           }
           lastDocBase = docBase;
         }
-        public void collect(int doc, float score) {}
+        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 MultiReaderHitCollector() {
+    s.search(q, new Collector() {
         int lastDocBase = -1;
         public void setNextReader(IndexReader reader, int docBase) {
           if (lastDocBase == -1) {
@@ -1681,7 +1683,8 @@
           }
           lastDocBase = docBase;
         }
-        public void collect(int doc, float score) {}
+        public void collect(int doc) {}
+        public void setScorer(Scorer scorer) {}
       });
     s.close();
     r.close();

Modified: lucene/java/trunk/src/test/org/apache/lucene/index/TestOmitTf.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/index/TestOmitTf.java?rev=764551&r1=764550&r2=764551&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/index/TestOmitTf.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/index/TestOmitTf.java Mon Apr 13 18:33:56 2009
@@ -17,6 +17,7 @@
  * limitations under the License.
  */
 
+import java.io.IOException;
 import java.util.Collection;
 
 import org.apache.lucene.util.LuceneTestCase;
@@ -27,7 +28,8 @@
 import org.apache.lucene.document.Field;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.MultiReaderHitCollector;
+import org.apache.lucene.search.Collector;
+import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Searcher;
 import org.apache.lucene.search.Similarity;
 import org.apache.lucene.search.TermQuery;
@@ -283,10 +285,15 @@
         
     searcher.search(q1,
                     new CountingHitCollector() {
-                      public final void collect(int doc, float score) {
+                      private Scorer scorer;
+                      public final void setScorer(Scorer scorer) {
+                        this.scorer = scorer;
+                      }
+                      public final void collect(int doc) throws IOException {
                         //System.out.println("Q1: Doc=" + doc + " score=" + score);
+                        float score = scorer.score();
                         assertTrue(score==1.0f);
-                        super.collect(doc, score);
+                        super.collect(doc);
                       }
                     });
     //System.out.println(CountingHitCollector.getCount());
@@ -294,10 +301,15 @@
         
     searcher.search(q2,
                     new CountingHitCollector() {
-                      public final void collect(int doc, float score) {
-                        //System.out.println("Q2: Doc=" + doc + " score=" + score);  
+                      private Scorer scorer;
+                      public final void setScorer(Scorer scorer) {
+                        this.scorer = scorer;
+                      }
+                      public final void collect(int doc) throws IOException {
+                        //System.out.println("Q2: Doc=" + doc + " score=" + score);
+                        float score = scorer.score();
                         assertTrue(score==1.0f+doc);
-                        super.collect(doc, score);
+                        super.collect(doc);
                       }
                     });
     //System.out.println(CountingHitCollector.getCount());
@@ -308,11 +320,16 @@
         
     searcher.search(q3,
                     new CountingHitCollector() {
-                      public final void collect(int doc, float score) {
+                      private Scorer scorer;
+                      public final void setScorer(Scorer scorer) {
+                        this.scorer = scorer;
+                      }
+                      public final void collect(int doc) throws IOException {
                         //System.out.println("Q1: Doc=" + doc + " score=" + score);
+                        float score = scorer.score();
                         assertTrue(score==1.0f);
                         assertFalse(doc%2==0);
-                        super.collect(doc, score);
+                        super.collect(doc);
                       }
                     });
     //System.out.println(CountingHitCollector.getCount());
@@ -320,11 +337,16 @@
         
     searcher.search(q4,
                     new CountingHitCollector() {
-                      public final void collect(int doc, float score) {
+                      private Scorer scorer;
+                      public final void setScorer(Scorer scorer) {
+                        this.scorer = scorer;
+                      }
+                      public final void collect(int doc) throws IOException {
+                        float score = scorer.score();
                         //System.out.println("Q1: Doc=" + doc + " score=" + score);
                         assertTrue(score==1.0f);
                         assertTrue(doc%2==0);
-                        super.collect(doc, score);
+                        super.collect(doc);
                       }
                     });
     //System.out.println(CountingHitCollector.getCount());
@@ -337,9 +359,9 @@
         
     searcher.search(bq,
                     new CountingHitCollector() {
-                      public final void collect(int doc, float score) {
+                      public final void collect(int doc) throws IOException {
                         //System.out.println("BQ: Doc=" + doc + " score=" + score);
-                        super.collect(doc, score);
+                        super.collect(doc);
                       }
                     });
     assertTrue(15 == CountingHitCollector.getCount());
@@ -348,12 +370,13 @@
     dir.close();
   }
      
-  public static class CountingHitCollector extends MultiReaderHitCollector {
+  public static class CountingHitCollector extends Collector {
     static int count=0;
     static int sum=0;
     private int docBase = -1;
     CountingHitCollector(){count=0;sum=0;}
-    public void collect(int doc, float score) {
+    public void setScorer(Scorer scorer) throws IOException {}
+    public void collect(int doc) throws IOException {
       count++;
       sum += doc + docBase;  // use it to avoid any possibility of being optimized away
     }

Modified: lucene/java/trunk/src/test/org/apache/lucene/search/CheckHits.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/search/CheckHits.java?rev=764551&r1=764550&r2=764551&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/search/CheckHits.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/search/CheckHits.java Mon Apr 13 18:33:56 2009
@@ -89,9 +89,10 @@
     }
     
     final Set actual = new TreeSet();
-    searcher.search(query, new MultiReaderHitCollector() {
-        private int base = -1;
-        public void collect(int doc, float score) {
+    searcher.search(query, new Collector() {
+        private int base = 0;
+        public void setScorer(Scorer scorer) throws IOException {}
+        public void collect(int doc) {
           actual.add(new Integer(doc + base));
         }
 
@@ -390,14 +391,22 @@
       checkExplanations(query);
       return super.search(query,filter,n,sort);
     }
+    /** @deprecated use {@link #search(Query, Collector)} instead. */
     public void search(Query query, HitCollector results) throws IOException {
+      search(query, new HitCollectorWrapper(results));
+    }
+    public void search(Query query, Collector results) throws IOException {
       checkExplanations(query);
-      super.search(query,results);
+      super.search(query, results);
     }
+    /** @deprecated use {@link #search(Query, Filter, Collector)} instead. */
     public void search(Query query, Filter filter,
         HitCollector results) throws IOException {
+      search(query, filter, new HitCollectorWrapper(results));
+    }
+    public void search(Query query, Filter filter, Collector results) throws IOException {
       checkExplanations(query);
-      super.search(query,filter, results);
+      super.search(query, filter, results);
     }
     public TopDocs search(Query query, Filter filter,
                           int n) throws IOException {
@@ -416,7 +425,7 @@
    *
    * @see CheckHits#verifyExplanation
    */
-  public static class ExplanationAsserter extends MultiReaderHitCollector {
+  public static class ExplanationAsserter extends Collector {
 
     /**
      * @deprecated
@@ -428,8 +437,9 @@
     Searcher s;
     String d;
     boolean deep;
-
-    private int base = -1;
+    
+    Scorer scorer;
+    private int base = 0;
 
     /** Constructs an instance which does shallow tests on the Explanation */
     public ExplanationAsserter(Query q, String defaultFieldName, Searcher s) {
@@ -441,8 +451,12 @@
       this.d = q.toString(defaultFieldName);
       this.deep=deep;
     }      
-
-    public void collect(int doc, float score) {
+    
+    public void setScorer(Scorer scorer) throws IOException {
+      this.scorer = scorer;     
+    }
+    
+    public void collect(int doc) throws IOException {
       Explanation exp = null;
       doc = doc + base;
       try {
@@ -454,7 +468,7 @@
       
       TestCase.assertNotNull("Explanation of [["+d+"]] for #"+doc+" is null",
                              exp);
-      verifyExplanation(d,doc,score,deep,exp);
+      verifyExplanation(d,doc,scorer.score(),deep,exp);
     }
     public void setNextReader(IndexReader reader, int docBase) {
       base = docBase;