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 ma...@apache.org on 2009/08/12 03:22:32 UTC

svn commit: r803339 [1/2] - in /lucene/java/trunk: ./ contrib/remote/src/java/org/apache/lucene/search/ src/java/org/apache/lucene/index/ src/java/org/apache/lucene/search/ src/java/org/apache/lucene/search/function/ src/java/org/apache/lucene/search/p...

Author: markrmiller
Date: Wed Aug 12 01:22:30 2009
New Revision: 803339

URL: http://svn.apache.org/viewvc?rev=803339&view=rev
Log:
LUCENE-1771: QueryWight back to Weight, but as abstract class rather than interface - explain now takes a Searcher and passes the sub reader that contains the doc if a top level reader is a multi reader.

Added:
    lucene/java/trunk/src/java/org/apache/lucene/util/ReaderUtil.java
Removed:
    lucene/java/trunk/src/java/org/apache/lucene/search/QueryWeight.java
    lucene/java/trunk/src/java/org/apache/lucene/search/QueryWeightWrapper.java
Modified:
    lucene/java/trunk/CHANGES.txt
    lucene/java/trunk/contrib/remote/src/java/org/apache/lucene/search/RemoteSearchable.java
    lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java
    lucene/java/trunk/src/java/org/apache/lucene/search/BooleanQuery.java
    lucene/java/trunk/src/java/org/apache/lucene/search/ConstantScoreQuery.java
    lucene/java/trunk/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java
    lucene/java/trunk/src/java/org/apache/lucene/search/ExactPhraseScorer.java
    lucene/java/trunk/src/java/org/apache/lucene/search/FilteredQuery.java
    lucene/java/trunk/src/java/org/apache/lucene/search/Hits.java
    lucene/java/trunk/src/java/org/apache/lucene/search/IndexSearcher.java
    lucene/java/trunk/src/java/org/apache/lucene/search/MatchAllDocsQuery.java
    lucene/java/trunk/src/java/org/apache/lucene/search/MultiPhraseQuery.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/PhraseQuery.java
    lucene/java/trunk/src/java/org/apache/lucene/search/PhraseScorer.java
    lucene/java/trunk/src/java/org/apache/lucene/search/Query.java
    lucene/java/trunk/src/java/org/apache/lucene/search/QueryWrapperFilter.java
    lucene/java/trunk/src/java/org/apache/lucene/search/Scorer.java
    lucene/java/trunk/src/java/org/apache/lucene/search/Searchable.java
    lucene/java/trunk/src/java/org/apache/lucene/search/Searcher.java
    lucene/java/trunk/src/java/org/apache/lucene/search/SloppyPhraseScorer.java
    lucene/java/trunk/src/java/org/apache/lucene/search/TermQuery.java
    lucene/java/trunk/src/java/org/apache/lucene/search/TermScorer.java
    lucene/java/trunk/src/java/org/apache/lucene/search/Weight.java
    lucene/java/trunk/src/java/org/apache/lucene/search/function/CustomScoreQuery.java
    lucene/java/trunk/src/java/org/apache/lucene/search/function/ValueSourceQuery.java
    lucene/java/trunk/src/java/org/apache/lucene/search/payloads/BoostingFunctionTermQuery.java
    lucene/java/trunk/src/java/org/apache/lucene/search/payloads/BoostingNearQuery.java
    lucene/java/trunk/src/java/org/apache/lucene/search/payloads/BoostingTermQuery.java
    lucene/java/trunk/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java
    lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanQuery.java
    lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanScorer.java
    lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanWeight.java
    lucene/java/trunk/src/test/org/apache/lucene/search/JustCompileSearch.java
    lucene/java/trunk/src/test/org/apache/lucene/search/QueryUtils.java
    lucene/java/trunk/src/test/org/apache/lucene/search/TestDisjunctionMaxQuery.java
    lucene/java/trunk/src/test/org/apache/lucene/search/TestTermScorer.java
    lucene/java/trunk/src/test/org/apache/lucene/search/spans/JustCompileSearchSpans.java
    lucene/java/trunk/src/test/org/apache/lucene/search/spans/TestNearSpansOrdered.java
    lucene/java/trunk/src/test/org/apache/lucene/search/spans/TestSpans.java

Modified: lucene/java/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/java/trunk/CHANGES.txt?rev=803339&r1=803338&r2=803339&view=diff
==============================================================================
--- lucene/java/trunk/CHANGES.txt (original)
+++ lucene/java/trunk/CHANGES.txt Wed Aug 12 01:22:30 2009
@@ -53,16 +53,15 @@
     which was unlikely done, because there is no possibility to change
     Lucene's FieldCache implementation.  (Grant Ingersoll, Uwe Schindler)
     
- 3. LUCENE-1630: Deprecate Weight in favor of QueryWeight: added
-    matching methods to Searcher to take QueryWeight and deprecated
-    those taking Weight.  If you have a Weight implementation, you can
-    turn it into a QueryWeight with QueryWeightWrapper (will be
-    removed in 3.0). All of the Weight-based methods were implemented
-    by calling the QueryWeight variants by wrapping the given Weight.
+ 3. LUCENE-1630, LUCENE-1771: Weight, previously an interface, is now an abstract 
+    class. Some of the method signatures have changed, but it should be fairly
+    easy to see what adjustments must be made to existing code to sync up
+    with the new API. You can find more detail in the API Changes section.
+    
     Going forward Searchable will be kept for convenience only and may
     be changed between minor releases without any deprecation
-    process. It is not recommended to implement it, but rather extend
-    Searcher.  (Shai Erera via Mike McCandless)
+    process. It is not recommended that you implement it, but rather extend
+    Searcher.  (Shai Erera, Chris Hostetter, Mark Miller via Mike McCandless)
 
  4. LUCENE-1422, LUCENE-1693: The new TokenStream API (see below) using
     Attributes has some backwards breaks in rare cases.
@@ -296,23 +295,22 @@
     NumericRangeQuery and its new indexing format for numeric or
     date values.  (Uwe Schindler)
     
-24. LUCENE-1630: Deprecate Weight in favor of QueryWeight, which adds
+24. LUCENE-1630, LUCENE-1771: Weight is now an abstract class, andd adds
     a scorer(IndexReader, boolean /* scoreDocsInOrder */, boolean /*
-    topScorer */) method instead of scorer(IndexReader) (now
-    deprecated). The new method is used by IndexSearcher to mate
-    between Collector and Scorer orderness of doc IDs. Some Scorers
-    (like BooleanScorer) are much more efficient if out-of-order
-    documents scoring is allowed by a Collector.  Collector must now
-    implement acceptsDocsOutOfOrder. If you write a Collector which
-    does not care about doc ID orderness, it is recommended that you
-    return true.  QueryWeight has the scoresDocsOutOfOrder method,
-    which by default returns false.  If you create a QueryWeight which
-    will score documents out of order if that's requested, you should
-    override that method to return true.  Also deprecated
-    BooleanQuery's setAllowDocsOutOfOrder and getAllowDocsOutOfOrder
-    as they are not needed anymore. BooleanQuery will now score docs
-    out of order when used with a Collector that can accept docs out
-    of order. (Shai Erera via Mike McCandless)
+    topScorer */) method instead of scorer(IndexReader). IndexSearcher uses 
+    this method to obtain a scorer matching the capabilities of the Collector 
+    wrt orderness of docIDs. Some Scorers (like BooleanScorer) are much more
+    efficient if out-of-order documents scoring is allowed by a Collector.  
+    Collector must now implement acceptsDocsOutOfOrder. If you write a 
+    Collector which does not care about doc ID orderness, it is recommended 
+    that you return true.  Weight has a scoresDocsOutOfOrder method, which by 
+    default returns false.  If you create a Weight which will score documents 
+    out of order if requested, you should override that method to return true. 
+    BooleanQuery's setAllowDocsOutOfOrder and getAllowDocsOutOfOrder have been 
+    deprecated as they are not needed anymore. BooleanQuery will now score docs 
+    out of order when used with a Collector that can accept docs out of order.
+    Finally, Weight#explain now also takes a Searcher.
+    (Shai Erera, Chris Hostetter, Mark Miller via Mike McCandless)
  	
 25. LUCENE-1466: Changed Tokenizer.input to be a CharStream; added
     CharFilter and MappingCharFilter, which allows chaining & mapping

Modified: lucene/java/trunk/contrib/remote/src/java/org/apache/lucene/search/RemoteSearchable.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/remote/src/java/org/apache/lucene/search/RemoteSearchable.java?rev=803339&r1=803338&r2=803339&view=diff
==============================================================================
--- lucene/java/trunk/contrib/remote/src/java/org/apache/lucene/search/RemoteSearchable.java (original)
+++ lucene/java/trunk/contrib/remote/src/java/org/apache/lucene/search/RemoteSearchable.java Wed Aug 12 01:22:30 2009
@@ -48,15 +48,10 @@
   /** @deprecated use {@link #search(Weight, Filter, Collector)} instead. */
   public void search(Weight weight, Filter filter, HitCollector results)
     throws IOException {
-    search(new QueryWeightWrapper(weight), filter, new HitCollectorWrapper(results));
+    local.search(weight, filter, results);
   }
 
   public void search(Weight weight, Filter filter, Collector results)
-      throws IOException {
-    search(new QueryWeightWrapper(weight), filter, results);
-  }
-
-  public void search(QueryWeight weight, Filter filter, Collector results)
   throws IOException {
     local.search(weight, filter, results);
   }
@@ -79,19 +74,10 @@
   }
 
   public TopDocs search(Weight weight, Filter filter, int n) throws IOException {
-    return search(new QueryWeightWrapper(weight), filter, n);
-  }
-  
-  public TopDocs search(QueryWeight weight, Filter filter, int n) throws IOException {
     return local.search(weight, filter, n);
   }
-
-  public TopFieldDocs search(Weight weight, Filter filter, int n, Sort sort)
-  throws IOException {
-    return search(new QueryWeightWrapper(weight), filter, n, sort);
-  }
   
-  public TopFieldDocs search(QueryWeight weight, Filter filter, int n, Sort sort)
+  public TopFieldDocs search(Weight weight, Filter filter, int n, Sort sort)
   throws IOException {
     return local.search (weight, filter, n, sort);
   }
@@ -109,10 +95,6 @@
   }
 
   public Explanation explain(Weight weight, int doc) throws IOException {
-    return explain(new QueryWeightWrapper(weight), doc);
-  }
-  
-  public Explanation explain(QueryWeight weight, int doc) throws IOException {
     return local.explain(weight, doc);
   }
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=803339&r1=803338&r2=803339&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java Wed Aug 12 01:22:30 2009
@@ -32,9 +32,9 @@
 import org.apache.lucene.document.Document;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.search.QueryWeight;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Similarity;
+import org.apache.lucene.search.Weight;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.ArrayUtil;
@@ -1005,7 +1005,7 @@
       Entry entry = (Entry) iter.next();
       Query query = (Query) entry.getKey();
       int limit = ((Integer) entry.getValue()).intValue();
-      QueryWeight weight = query.queryWeight(searcher);
+      Weight weight = query.weight(searcher);
       Scorer scorer = weight.scorer(reader, true, false);
       if (scorer != null) {
         while(true)  {

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=803339&r1=803338&r2=803339&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 Wed Aug 12 01:22:30 2009
@@ -179,11 +179,9 @@
    * <p>NOTE: this API and implementation is subject to
    * change suddenly in the next release.</p>
    */
-  protected class BooleanWeight extends QueryWeight {
+  protected class BooleanWeight extends Weight {
     /** The Similarity implementation. */
     protected Similarity similarity;
-
-    /** The Weights for our subqueries, in 1-1 correspondence with clauses */
     protected ArrayList weights;
 
     public BooleanWeight(Searcher searcher)
@@ -192,7 +190,7 @@
       weights = new ArrayList(clauses.size());
       for (int i = 0 ; i < clauses.size(); i++) {
         BooleanClause c = (BooleanClause)clauses.get(i);
-        weights.add(c.getQuery().createQueryWeight(searcher));
+        weights.add(c.getQuery().createWeight(searcher));
       }
     }
 
@@ -203,7 +201,7 @@
       float sum = 0.0f;
       for (int i = 0 ; i < weights.size(); i++) {
         BooleanClause c = (BooleanClause)clauses.get(i);
-        QueryWeight w = (QueryWeight)weights.get(i);
+        Weight w = (Weight)weights.get(i);
         // call sumOfSquaredWeights for all clauses in case of side effects
         float s = w.sumOfSquaredWeights();         // sum sub weights
         if (!c.isProhibited())
@@ -220,13 +218,13 @@
     public void normalize(float norm) {
       norm *= getBoost();                         // incorporate boost
       for (Iterator iter = weights.iterator(); iter.hasNext();) {
-        QueryWeight w = (QueryWeight) iter.next();
+        Weight w = (Weight) iter.next();
         // normalize all clauses, (even if prohibited in case of side affects)
         w.normalize(norm);
       }
     }
 
-    public Explanation explain(IndexReader reader, int doc)
+    public Explanation explain(Searcher searcher, IndexReader reader, int doc)
       throws IOException {
       final int minShouldMatch =
         BooleanQuery.this.getMinimumNumberShouldMatch();
@@ -238,12 +236,12 @@
       boolean fail = false;
       int shouldMatchCount = 0;
       for (Iterator wIter = weights.iterator(), cIter = clauses.iterator(); wIter.hasNext();) {
-        QueryWeight w = (QueryWeight) wIter.next();
+        Weight w = (Weight) wIter.next();
         BooleanClause c = (BooleanClause) cIter.next();
         if (w.scorer(reader, true, true) == null) {
           continue;
         }
-        Explanation e = w.explain(reader, doc);
+        Explanation e = w.explain(searcher, reader, doc);
         if (!c.isProhibited()) maxCoord++;
         if (e.isMatch()) {
           if (!c.isProhibited()) {
@@ -303,7 +301,7 @@
       List prohibited = new ArrayList();
       List optional = new ArrayList();
       for (Iterator wIter = weights.iterator(), cIter = clauses.iterator(); wIter.hasNext();) {
-        QueryWeight w = (QueryWeight) wIter.next();
+        Weight w = (Weight) wIter.next();
         BooleanClause c = (BooleanClause) cIter.next();
         Scorer subScorer = w.scorer(reader, true, false);
         if (subScorer == null) {
@@ -364,7 +362,7 @@
    * Whether hit docs may be collected out of docid order.
    * 
    * @deprecated this will not be needed anymore, as
-   *             {@link QueryWeight#scoresDocsOutOfOrder()} is used.
+   *             {@link Weight#scoresDocsOutOfOrder()} is used.
    */
   private static boolean allowDocsOutOfOrder = true;
 
@@ -391,7 +389,7 @@
    * </p>
    * 
    * @deprecated this is not needed anymore, as
-   *             {@link QueryWeight#scoresDocsOutOfOrder()} is used.
+   *             {@link Weight#scoresDocsOutOfOrder()} is used.
    */
   public static void setAllowDocsOutOfOrder(boolean allow) {
     allowDocsOutOfOrder = allow;
@@ -402,7 +400,7 @@
    * 
    * @see #setAllowDocsOutOfOrder(boolean)
    * @deprecated this is not needed anymore, as
-   *             {@link QueryWeight#scoresDocsOutOfOrder()} is used.
+   *             {@link Weight#scoresDocsOutOfOrder()} is used.
    */
   public static boolean getAllowDocsOutOfOrder() {
     return allowDocsOutOfOrder;
@@ -422,7 +420,7 @@
 	return getAllowDocsOutOfOrder();
   }
 
-  public QueryWeight createQueryWeight(Searcher searcher) throws IOException {
+  public Weight createWeight(Searcher searcher) throws IOException {
     return new BooleanWeight(searcher);
   }
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/ConstantScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/ConstantScoreQuery.java?rev=803339&r1=803338&r2=803339&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/ConstantScoreQuery.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/ConstantScoreQuery.java Wed Aug 12 01:22:30 2009
@@ -50,7 +50,7 @@
     // but may not be OK for highlighting
   }
 
-  protected class ConstantWeight extends QueryWeight {
+  protected class ConstantWeight extends Weight {
     private Similarity similarity;
     private float queryNorm;
     private float queryWeight;
@@ -81,9 +81,9 @@
       return new ConstantScorer(similarity, reader, this);
     }
 
-    public Explanation explain(IndexReader reader, int doc) throws IOException {
-
-      ConstantScorer cs = (ConstantScorer) scorer(reader, true, false);
+    public Explanation explain(Searcher searcher, IndexReader reader, int doc) throws IOException {
+      
+      ConstantScorer cs = new ConstantScorer(similarity, reader, this);
       boolean exists = cs.docIdSetIterator.advance(doc) == doc;
 
       ComplexExplanation result = new ComplexExplanation();
@@ -110,7 +110,7 @@
     final float theScore;
     int doc = -1;
 
-    public ConstantScorer(Similarity similarity, IndexReader reader, QueryWeight w) throws IOException {
+    public ConstantScorer(Similarity similarity, IndexReader reader, Weight w) throws IOException {
       super(similarity);
       theScore = w.getValue();
       DocIdSet docIdSet = filter.getDocIdSet(reader);
@@ -162,7 +162,7 @@
     }
   }
 
-  public QueryWeight createQueryWeight(Searcher searcher) {
+  public Weight createWeight(Searcher searcher) {
     return new ConstantScoreQuery.ConstantWeight(searcher);
   }
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java?rev=803339&r1=803338&r2=803339&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java Wed Aug 12 01:22:30 2009
@@ -92,18 +92,18 @@
    * <p>NOTE: this API and implementation is subject to
    * change suddenly in the next release.</p>
    */
-  protected class DisjunctionMaxWeight extends QueryWeight {
+  protected class DisjunctionMaxWeight extends Weight {
     /** The Similarity implementation. */
     protected Similarity similarity;
 
     /** The Weights for our subqueries, in 1-1 correspondence with disjuncts */
-    protected ArrayList weights = new ArrayList();
+    protected ArrayList weights = new ArrayList();  // The Weight's for our subqueries, in 1-1 correspondence with disjuncts
 
     /* Construct the Weight for this Query searched by searcher.  Recursively construct subquery weights. */
     public DisjunctionMaxWeight(Searcher searcher) throws IOException {
       this.similarity = searcher.getSimilarity();
       for (Iterator iter = disjuncts.iterator(); iter.hasNext();) {
-        weights.add(((Query) iter.next()).createQueryWeight(searcher));
+        weights.add(((Query) iter.next()).createWeight(searcher));
       }
     }
 
@@ -117,7 +117,7 @@
     public float sumOfSquaredWeights() throws IOException {
       float max = 0.0f, sum = 0.0f;
       for (Iterator iter = weights.iterator(); iter.hasNext();) {
-        float sub = ((QueryWeight) iter.next()).sumOfSquaredWeights();
+        float sub = ((Weight) iter.next()).sumOfSquaredWeights();
         sum += sub;
         max = Math.max(max, sub);
         
@@ -130,7 +130,7 @@
     public void normalize(float norm) {
       norm *= getBoost();  // Incorporate our boost
       for (Iterator iter = weights.iterator(); iter.hasNext();) {
-        ((QueryWeight) iter.next()).normalize(norm);
+        ((Weight) iter.next()).normalize(norm);
       }
     }
 
@@ -140,7 +140,7 @@
       Scorer[] scorers = new Scorer[weights.size()];
       int idx = 0;
       for (Iterator iter = weights.iterator(); iter.hasNext();) {
-        QueryWeight w = (QueryWeight) iter.next();
+        Weight w = (Weight) iter.next();
         Scorer subScorer = w.scorer(reader, true, false);
         if (subScorer != null && subScorer.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
           scorers[idx++] = subScorer;
@@ -152,13 +152,13 @@
     }
 
     /* Explain the score we computed for doc */
-    public Explanation explain(IndexReader reader, int doc) throws IOException {
-      if (disjuncts.size() == 1) return ((QueryWeight) weights.get(0)).explain(reader,doc);
+    public Explanation explain(Searcher searcher, IndexReader reader, int doc) throws IOException {
+      if (disjuncts.size() == 1) return ((Weight) weights.get(0)).explain(searcher, reader,doc);
       ComplexExplanation result = new ComplexExplanation();
       float max = 0.0f, sum = 0.0f;
       result.setDescription(tieBreakerMultiplier == 0.0f ? "max of:" : "max plus " + tieBreakerMultiplier + " times others of:");
       for (Iterator iter = weights.iterator(); iter.hasNext();) {
-        Explanation e = ((QueryWeight) iter.next()).explain(reader, doc);
+        Explanation e = ((Weight) iter.next()).explain(searcher, reader, doc);
         if (e.isMatch()) {
           result.setMatch(Boolean.TRUE);
           result.addDetail(e);
@@ -172,8 +172,8 @@
     
   }  // end of DisjunctionMaxWeight inner class
 
-  /* Create the QueryWeight used to score us */
-  public QueryWeight createQueryWeight(Searcher searcher) throws IOException {
+  /* Create the Weight used to score us */
+  public Weight createWeight(Searcher searcher) throws IOException {
     return new DisjunctionMaxWeight(searcher);
   }
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/ExactPhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/ExactPhraseScorer.java?rev=803339&r1=803338&r2=803339&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/ExactPhraseScorer.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/ExactPhraseScorer.java Wed Aug 12 01:22:30 2009
@@ -22,7 +22,7 @@
 
 final class ExactPhraseScorer extends PhraseScorer {
 
-  ExactPhraseScorer(QueryWeight weight, TermPositions[] tps, int[] offsets,
+  ExactPhraseScorer(Weight weight, TermPositions[] tps, int[] offsets,
       Similarity similarity, byte[] norms) {
     super(weight, tps, offsets, similarity, norms);
   }

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/FilteredQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/FilteredQuery.java?rev=803339&r1=803338&r2=803339&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/FilteredQuery.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/FilteredQuery.java Wed Aug 12 01:22:30 2009
@@ -58,10 +58,10 @@
    * Returns a Weight that applies the filter to the enclosed query's Weight.
    * This is accomplished by overriding the Scorer returned by the Weight.
    */
-  public QueryWeight createQueryWeight(final Searcher searcher) throws IOException {
-    final QueryWeight weight = query.createQueryWeight (searcher);
+  public Weight createWeight(final Searcher searcher) throws IOException {
+    final Weight weight = query.createWeight (searcher);
     final Similarity similarity = query.getSimilarity(searcher);
-    return new QueryWeight() {
+    return new Weight() {
       private float value;
         
       // pass these methods through to enclosed query's weight
@@ -73,8 +73,8 @@
         weight.normalize(v);
         value = weight.getValue() * getBoost();
       }
-      public Explanation explain (IndexReader ir, int i) throws IOException {
-        Explanation inner = weight.explain (ir, i);
+      public Explanation explain (Searcher searcher, IndexReader ir, int i) throws IOException {
+        Explanation inner = weight.explain (searcher, ir, i);
         if (getBoost()!=1) {
           Explanation preBoost = inner;
           inner = new Explanation(inner.getValue()*getBoost(),"product of:");

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/Hits.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/Hits.java?rev=803339&r1=803338&r2=803339&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/Hits.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/Hits.java Wed Aug 12 01:22:30 2009
@@ -53,7 +53,7 @@
  * </pre>
  */
 public final class Hits {
-  private QueryWeight weight;
+  private Weight weight;
   private Searcher searcher;
   private Filter filter = null;
   private Sort sort = null;
@@ -73,7 +73,7 @@
   boolean debugCheckedForDeletions = false; // for test purposes.
 
   Hits(Searcher s, Query q, Filter f) throws IOException {
-    weight = q.queryWeight(s);
+    weight = q.weight(s);
     searcher = s;
     filter = f;
     nDeletions = countDeletions(s);
@@ -82,7 +82,7 @@
   }
 
   Hits(Searcher s, Query q, Filter f, Sort o) throws IOException {
-    weight = q.queryWeight(s);
+    weight = q.weight(s);
     searcher = s;
     filter = f;
     sort = o;

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=803339&r1=803338&r2=803339&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 Wed Aug 12 01:22:30 2009
@@ -27,6 +27,7 @@
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.ReaderUtil;
 
 /** Implements search over a single IndexReader.
  *
@@ -121,15 +122,7 @@
   }
 
   protected void gatherSubReaders(List allSubReaders, IndexReader r) {
-    IndexReader[] subReaders = r.getSequentialSubReaders();
-    if (subReaders == null) {
-      // Add the reader itself, and do not recurse
-      allSubReaders.add(r);
-    } else {
-      for (int i = 0; i < subReaders.length; i++) {
-        gatherSubReaders(allSubReaders, subReaders[i]);
-      }
-    }
+    ReaderUtil.gatherSubReaders(allSubReaders, r);
   }
 
   /** Return the {@link IndexReader} this searches. */
@@ -169,7 +162,7 @@
   }
 
   // inherit javadoc
-  public TopDocs search(QueryWeight weight, Filter filter, final int nDocs) throws IOException {
+  public TopDocs search(Weight weight, Filter filter, final int nDocs) throws IOException {
 
     if (nDocs <= 0) {
       throw new IllegalArgumentException("nDocs must be > 0");
@@ -180,22 +173,22 @@
     return collector.topDocs();
   }
 
-  public TopFieldDocs search(QueryWeight weight, Filter filter,
+  public TopFieldDocs search(Weight weight, Filter filter,
       final int nDocs, Sort sort) throws IOException {
     return search(weight, filter, nDocs, sort, true);
   }
 
   /**
-   * Just like {@link #search(QueryWeight, Filter, int, Sort)}, but you choose
+   * Just like {@link #search(Weight, Filter, int, Sort)}, but you choose
    * whether or not the fields in the returned {@link FieldDoc} instances should
    * be set by specifying fillFields.<br>
    * <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(QueryWeight, Filter, Collector)} and pass in a
+   * {@link #search(Weight, Filter, Collector)} and pass in a
    * {@link TopFieldCollector} instance.
    */
-  public TopFieldDocs search(QueryWeight weight, Filter filter, final int nDocs,
+  public TopFieldDocs search(Weight weight, Filter filter, final int nDocs,
                              Sort sort, boolean fillFields)
       throws IOException {
     
@@ -242,7 +235,7 @@
     return (TopFieldDocs) collector.topDocs();
   }
 
-  public void search(QueryWeight weight, Filter filter, Collector collector)
+  public void search(Weight weight, Filter filter, Collector collector)
       throws IOException {
     
     if (filter == null) {
@@ -261,7 +254,7 @@
     }
   }
 
-  private void searchWithFilter(IndexReader reader, QueryWeight weight,
+  private void searchWithFilter(IndexReader reader, Weight weight,
       final Filter filter, final Collector collector) throws IOException {
 
     assert filter != null;
@@ -316,8 +309,11 @@
     return query;
   }
 
-  public Explanation explain(QueryWeight weight, int doc) throws IOException {
-    return weight.explain(reader, doc);
+  public Explanation explain(Weight weight, int doc) throws IOException {
+    int n = ReaderUtil.subIndex(doc, docStarts);
+    int deBasedDoc = doc - docStarts[n];
+    
+    return weight.explain(this, subReaders[n], deBasedDoc);
   }
 
   private boolean fieldSortDoTrackScores;

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/MatchAllDocsQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/MatchAllDocsQuery.java?rev=803339&r1=803338&r2=803339&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/MatchAllDocsQuery.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/MatchAllDocsQuery.java Wed Aug 12 01:22:30 2009
@@ -49,7 +49,7 @@
     final byte[] norms;
     private int doc = -1;
     
-    MatchAllScorer(IndexReader reader, Similarity similarity, QueryWeight w,
+    MatchAllScorer(IndexReader reader, Similarity similarity, Weight w,
         byte[] norms) throws IOException {
       super(similarity);
       this.termDocs = reader.termDocs(null);
@@ -93,7 +93,7 @@
     }
   }
 
-  private class MatchAllDocsWeight extends QueryWeight {
+  private class MatchAllDocsWeight extends Weight {
     private Similarity similarity;
     private float queryWeight;
     private float queryNorm;
@@ -129,7 +129,7 @@
           normsField != null ? reader.norms(normsField) : null);
     }
 
-    public Explanation explain(IndexReader reader, int doc) {
+    public Explanation explain(Searcher searcher, IndexReader reader, int doc) {
       // explain query weight
       Explanation queryExpl = new ComplexExplanation
         (true, getValue(), "MatchAllDocsQuery, product of:");
@@ -142,7 +142,7 @@
     }
   }
 
-  public QueryWeight createQueryWeight(Searcher searcher) {
+  public Weight createWeight(Searcher searcher) {
     return new MatchAllDocsWeight(searcher);
   }
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/MultiPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/MultiPhraseQuery.java?rev=803339&r1=803338&r2=803339&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/MultiPhraseQuery.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/MultiPhraseQuery.java Wed Aug 12 01:22:30 2009
@@ -123,7 +123,7 @@
   }
 
 
-  private class MultiPhraseWeight extends QueryWeight {
+  private class MultiPhraseWeight extends Weight {
     private Similarity similarity;
     private float value;
     private float idf;
@@ -186,7 +186,7 @@
                                       slop, reader.norms(field));
     }
 
-    public Explanation explain(IndexReader reader, int doc)
+    public Explanation explain(Searcher searcher, IndexReader reader, int doc)
       throws IOException {
       ComplexExplanation result = new ComplexExplanation();
       result.setDescription("weight("+getQuery()+" in "+doc+"), product of:");
@@ -265,7 +265,7 @@
     }
   }
 
-  public QueryWeight createQueryWeight(Searcher searcher) throws IOException {
+  public Weight createWeight(Searcher searcher) throws IOException {
     return new MultiPhraseWeight(searcher);
   }
 

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=803339&r1=803338&r2=803339&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 Wed Aug 12 01:22:30 2009
@@ -22,6 +22,7 @@
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.util.ReaderUtil;
 
 import java.io.IOException;
 import java.util.HashMap;
@@ -94,19 +95,19 @@
         throw new UnsupportedOperationException();
     }
 
-    public Explanation explain(QueryWeight weight,int doc) {
+    public Explanation explain(Weight weight,int doc) {
       throw new UnsupportedOperationException();
     }
 
-    public void search(QueryWeight weight, Filter filter, Collector results) {
+    public void search(Weight weight, Filter filter, Collector results) {
       throw new UnsupportedOperationException();
     }
     
-    public TopDocs search(QueryWeight weight,Filter filter,int n) {
+    public TopDocs search(Weight weight,Filter filter,int n) {
       throw new UnsupportedOperationException();
     }
 
-    public TopFieldDocs search(QueryWeight weight,Filter filter,int n,Sort sort) {
+    public TopFieldDocs search(Weight weight,Filter filter,int n,Sort sort) {
       throw new UnsupportedOperationException();
     }
   }
@@ -164,25 +165,7 @@
   /** Returns index of the searcher for document <code>n</code> in the array
    * used to construct this searcher. */
   public int subSearcher(int n) {                 // find searcher for doc n:
-    // replace w/ call to Arrays.binarySearch in Java 1.2
-    int lo = 0;					  // search starts array
-    int hi = searchables.length - 1;		  // for first element less
-						  // than n, return its index
-    while (hi >= lo) {
-      int mid = (lo + hi) >>> 1;
-      int midValue = starts[mid];
-      if (n < midValue)
-	hi = mid - 1;
-      else if (n > midValue)
-	lo = mid + 1;
-      else {                                      // found a match
-        while (mid+1 < searchables.length && starts[mid+1] == midValue) {
-          mid++;                                  // scan to last match
-        }
-	return mid;
-      }
-    }
-    return hi;
+    return ReaderUtil.subIndex(n, starts);
   }
 
   /** Returns the document number of document <code>n</code> within its
@@ -195,7 +178,7 @@
     return maxDoc;
   }
 
-  public TopDocs search(QueryWeight weight, Filter filter, int nDocs)
+  public TopDocs search(Weight weight, Filter filter, int nDocs)
       throws IOException {
 
     HitQueue hq = new HitQueue(nDocs, false);
@@ -222,7 +205,7 @@
     return new TopDocs(totalHits, scoreDocs, maxScore);
   }
 
-  public TopFieldDocs search (QueryWeight weight, Filter filter, int n, Sort sort)
+  public TopFieldDocs search (Weight weight, Filter filter, int n, Sort sort)
   throws IOException {
     FieldDocSortedHitQueue hq = null;
     int totalHits = 0;
@@ -264,7 +247,7 @@
   }
 
   // inherit javadoc
-  public void search(QueryWeight weight, Filter filter, final Collector collector)
+  public void search(Weight weight, Filter filter, final Collector collector)
   throws IOException {
     for (int i = 0; i < searchables.length; i++) {
       
@@ -297,7 +280,7 @@
     return queries[0].combine(queries);
   }
 
-  public Explanation explain(QueryWeight weight, int doc) throws IOException {
+  public Explanation explain(Weight weight, int doc) throws IOException {
     int i = subSearcher(doc);			  // find searcher index
     return searchables[i].explain(weight, doc - starts[i]); // dispatch to searcher
   }
@@ -317,7 +300,7 @@
    *
    * @return rewritten queries
    */
-  protected QueryWeight createQueryWeight(Query original) throws IOException {
+  protected Weight createWeight(Query original) throws IOException {
     // step 1
     Query rewrittenQuery = rewrite(original);
 
@@ -345,7 +328,7 @@
     int numDocs = maxDoc();
     CachedDfSource cacheSim = new CachedDfSource(dfMap, numDocs, getSimilarity());
 
-    return rewrittenQuery.queryWeight(cacheSim);
+    return rewrittenQuery.weight(cacheSim);
   }
 
 }

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=803339&r1=803338&r2=803339&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 Wed Aug 12 01:22:30 2009
@@ -52,7 +52,7 @@
    * Searchable, waits for each search to complete and merge
    * the results back together.
    */
-  public TopDocs search(QueryWeight weight, Filter filter, int nDocs)
+  public TopDocs search(Weight weight, Filter filter, int nDocs)
     throws IOException {
     HitQueue hq = new HitQueue(nDocs, false);
     int totalHits = 0;
@@ -97,7 +97,7 @@
    * Searchable, waits for each search to complete and merges
    * the results back together.
    */
-  public TopFieldDocs search(QueryWeight weight, Filter filter, int nDocs, Sort sort)
+  public TopFieldDocs search(Weight weight, Filter filter, int nDocs, Sort sort)
     throws IOException {
     // don't specify the fields - we'll wait to do this until we get results
     FieldDocSortedHitQueue hq = new FieldDocSortedHitQueue (null, nDocs);
@@ -153,7 +153,7 @@
   * 
   * @todo parallelize this one too
   */
-  public void search(QueryWeight weight, Filter filter, final Collector collector)
+  public void search(Weight weight, Filter filter, final Collector collector)
    throws IOException {
    for (int i = 0; i < searchables.length; i++) {
 
@@ -194,7 +194,7 @@
 class MultiSearcherThread extends Thread {
 
   private Searchable searchable;
-  private QueryWeight weight;
+  private Weight weight;
   private Filter filter;
   private int nDocs;
   private TopDocs docs;
@@ -204,7 +204,7 @@
   private IOException ioe;
   private Sort sort;
 
-  public MultiSearcherThread(Searchable searchable, QueryWeight weight, Filter filter,
+  public MultiSearcherThread(Searchable searchable, Weight weight, Filter filter,
       int nDocs, HitQueue hq, int i, int[] starts, String name) {
     super(name);
     this.searchable = searchable;
@@ -216,7 +216,7 @@
     this.starts = starts;
   }
 
-  public MultiSearcherThread(Searchable searchable, QueryWeight weight,
+  public MultiSearcherThread(Searchable searchable, Weight weight,
       Filter filter, int nDocs, FieldDocSortedHitQueue hq, Sort sort, int i,
       int[] starts, String name) {
     super(name);

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/PhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/PhraseQuery.java?rev=803339&r1=803338&r2=803339&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/PhraseQuery.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/PhraseQuery.java Wed Aug 12 01:22:30 2009
@@ -106,7 +106,7 @@
       return result;
   }
 
-  private class PhraseWeight extends QueryWeight {
+  private class PhraseWeight extends Weight {
     private Similarity similarity;
     private float value;
     private float idf;
@@ -158,7 +158,7 @@
 
     }
 
-    public Explanation explain(IndexReader reader, int doc)
+    public Explanation explain(Searcher searcher, IndexReader reader, int doc)
       throws IOException {
 
       Explanation result = new Explanation();
@@ -241,12 +241,12 @@
     }
   }
 
-  public QueryWeight createQueryWeight(Searcher searcher) throws IOException {
+  public Weight createWeight(Searcher searcher) throws IOException {
     if (terms.size() == 1) {			  // optimize one-term case
       Term term = (Term)terms.get(0);
       Query termQuery = new TermQuery(term);
       termQuery.setBoost(getBoost());
-      return termQuery.createQueryWeight(searcher);
+      return termQuery.createWeight(searcher);
     }
     return new PhraseWeight(searcher);
   }

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/PhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/PhraseScorer.java?rev=803339&r1=803338&r2=803339&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/PhraseScorer.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/PhraseScorer.java Wed Aug 12 01:22:30 2009
@@ -32,7 +32,7 @@
  * means a match. 
  */
 abstract class PhraseScorer extends Scorer {
-  private QueryWeight weight;
+  private Weight weight;
   protected byte[] norms;
   protected float value;
 
@@ -43,7 +43,7 @@
 
   private float freq; //prhase frequency in current doc as computed by phraseFreq().
 
-  PhraseScorer(QueryWeight weight, TermPositions[] tps, int[] offsets,
+  PhraseScorer(Weight weight, TermPositions[] tps, int[] offsets,
       Similarity similarity, byte[] norms) {
     super(similarity);
     this.norms = norms;

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/Query.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/Query.java?rev=803339&r1=803338&r2=803339&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/Query.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/Query.java Wed Aug 12 01:22:30 2009
@@ -86,45 +86,17 @@
    * 
    * <p>
    * Only implemented by primitive queries, which re-write to themselves.
-   * @deprecated use {@link #createQueryWeight(Searcher)} instead.
    */
   protected Weight createWeight(Searcher searcher) throws IOException {
     throw new UnsupportedOperationException();
   }
 
   /**
-   * Expert: Constructs an appropriate {@link QueryWeight} implementation for
-   * this query.
-   * <p>
-   * Only implemented by primitive queries, which re-write to themselves.
-   * <p>
-   * <b>NOTE:</b> in 3.0 this method will throw
-   * {@link UnsupportedOperationException}. It is implemented now by calling
-   * {@link #createWeight(Searcher)} for backwards compatibility, for
-   * {@link Query} implementations that did not override it yet (but did
-   * override {@link #createWeight(Searcher)}).
-   */
-  // TODO (3.0): change to throw UnsupportedOperationException.
-  public QueryWeight createQueryWeight(Searcher searcher) throws IOException {
-    return new QueryWeightWrapper(createWeight(searcher));
-  }
-
-  /**
    * Expert: Constructs and initializes a Weight for a top-level query.
-   * 
-   * @deprecated use {@link #queryWeight(Searcher)} instead.
    */
   public Weight weight(Searcher searcher) throws IOException {
-    return queryWeight(searcher);
-  }
-
-  /**
-   * Expert: Constructs and initializes a {@link QueryWeight} for a top-level
-   * query.
-   */
-  public QueryWeight queryWeight(Searcher searcher) throws IOException {
     Query query = searcher.rewrite(this);
-    QueryWeight weight = query.createQueryWeight(searcher);
+    Weight weight = query.createWeight(searcher);
     float sum = weight.sumOfSquaredWeights();
     float norm = getSimilarity(searcher).queryNorm(sum);
     weight.normalize(norm);

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/QueryWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/QueryWrapperFilter.java?rev=803339&r1=803338&r2=803339&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/QueryWrapperFilter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/QueryWrapperFilter.java Wed Aug 12 01:22:30 2009
@@ -69,7 +69,7 @@
   }
   
   public DocIdSet getDocIdSet(final IndexReader reader) throws IOException {
-    final QueryWeight weight = query.queryWeight(new IndexSearcher(reader));
+    final Weight weight = query.createWeight(new IndexSearcher(reader));
     return new DocIdSet() {
       public DocIdSetIterator iterator() throws IOException {
         return weight.scorer(reader, true, false);

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=803339&r1=803338&r2=803339&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 Wed Aug 12 01:22:30 2009
@@ -127,7 +127,7 @@
    * @param doc The document number for the explanation.
    *
    * @deprecated Please use {@link IndexSearcher#explain}
-   * or {@link QueryWeight#explain} instead.
+   * or {@link Weight#explain} instead.
    */
   public abstract Explanation explain(int doc) 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=803339&r1=803338&r2=803339&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 Wed Aug 12 01:22:30 2009
@@ -58,7 +58,7 @@
    * @param filter if non-null, used to permit documents to be collected.
    * @param results to receive hits
    * @throws BooleanQuery.TooManyClauses
-   * @deprecated use {@link #search(QueryWeight, Filter, Collector)} instead.
+   * @deprecated use {@link #search(Weight, Filter, Collector)} instead.
    */
   void search(Weight weight, Filter filter, HitCollector results)
   throws IOException;
@@ -82,33 +82,9 @@
    * @param collector
    *          to receive hits
    * @throws BooleanQuery.TooManyClauses
-   * 
-   * @deprecated use {@link #search(QueryWeight, Filter, Collector)} instead.
    */
   void search(Weight weight, Filter filter, Collector collector) 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(QueryWeight 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
    * like {@link Hits}.
@@ -141,20 +117,9 @@
    * <p>Applications should usually call {@link Searcher#search(Query)} or
    * {@link Searcher#search(Query,Filter)} instead.
    * @throws BooleanQuery.TooManyClauses
-   * @deprecated use {@link #search(QueryWeight, Filter, int)} instead.
+   * @deprecated use {@link #search(Weight, Filter, int)} instead.
    */
   TopDocs search(Weight weight, Filter filter, int n) throws IOException;
-  
-  /** Expert: Low-level search implementation.  Finds the top <code>n</code>
-   * hits for <code>query</code>, applying <code>filter</code> if non-null.
-   *
-   * <p>Called by {@link Hits}.
-   *
-   * <p>Applications should usually call {@link Searcher#search(Query)} or
-   * {@link Searcher#search(Query,Filter)} instead.
-   * @throws BooleanQuery.TooManyClauses
-   */
-  TopDocs search(QueryWeight weight, Filter filter, int n) throws IOException;
 
   /** Expert: Returns the stored fields of document <code>i</code>.
    * Called by {@link HitCollector} implementations.
@@ -202,22 +167,9 @@
    * entire index.
    * <p>Applications should call {@link Searcher#explain(Query, int)}.
    * @throws BooleanQuery.TooManyClauses
-   * @deprecated use {@link #explain(QueryWeight, int)} instead.
+   * @deprecated use {@link #explain(Weight, int)} instead.
    */
   Explanation explain(Weight weight, int doc) throws IOException;
-  
-  /** Expert: low-level implementation method
-   * Returns an Explanation that describes how <code>doc</code> scored against
-   * <code>weight</code>.
-   *
-   * <p>This is intended to be used in developing Similarity implementations,
-   * and, for good performance, should not be displayed with every hit.
-   * Computing an explanation is as expensive as executing the query over the
-   * entire index.
-   * <p>Applications should call {@link Searcher#explain(Query, int)}.
-   * @throws BooleanQuery.TooManyClauses
-   */
-  Explanation explain(QueryWeight weight, int doc) throws IOException;
 
   /** Expert: Low-level search implementation with arbitrary sorting.  Finds
    * the top <code>n</code> hits for <code>query</code>, applying
@@ -228,22 +180,8 @@
    * Searcher#search(Query,Filter,Sort)} instead.
    * 
    * @throws BooleanQuery.TooManyClauses
-   * @deprecated use {@link #search(QueryWeight, Filter, int, Sort)} instead.
    */
   TopFieldDocs search(Weight weight, Filter filter, int n, Sort sort)
   throws IOException;
-  
-  /** 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
-   * <code>sort</code>.
-   *
-   * <p>Applications should usually call {@link
-   * Searcher#search(Query,Filter,Sort)} instead.
-   * 
-   * @throws BooleanQuery.TooManyClauses
-   */
-  TopFieldDocs search(QueryWeight weight, Filter filter, int n, Sort sort)
-  throws IOException;
 
 }

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=803339&r1=803338&r2=803339&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 Wed Aug 12 01:22:30 2009
@@ -89,7 +89,7 @@
    */
   public TopFieldDocs search(Query query, Filter filter, int n,
                              Sort sort) throws IOException {
-    return search(createQueryWeight(query), filter, n, sort);
+    return search(createWeight(query), filter, n, sort);
   }
 
   /** Lower-level search API.
@@ -109,7 +109,7 @@
    */
   public void search(Query query, HitCollector results)
     throws IOException {
-    search(createQueryWeight(query), null, new HitCollectorWrapper(results));
+    search(createWeight(query), null, new HitCollectorWrapper(results));
   }
 
   /** Lower-level search API.
@@ -127,7 +127,7 @@
   */
  public void search(Query query, Collector results)
    throws IOException {
-   search(createQueryWeight(query), null, results);
+   search(createWeight(query), null, results);
  }
 
   /** Lower-level search API.
@@ -149,7 +149,7 @@
    */
   public void search(Query query, Filter filter, HitCollector results)
     throws IOException {
-    search(createQueryWeight(query), filter, new HitCollectorWrapper(results));
+    search(createWeight(query), filter, new HitCollectorWrapper(results));
   }
   
   /** Lower-level search API.
@@ -170,7 +170,7 @@
    */
   public void search(Query query, Filter filter, Collector results)
   throws IOException {
-    search(createQueryWeight(query), filter, results);
+    search(createWeight(query), filter, results);
   }
 
   /** Finds the top <code>n</code>
@@ -180,7 +180,7 @@
    */
   public TopDocs search(Query query, Filter filter, int n)
     throws IOException {
-    return search(createQueryWeight(query), filter, n);
+    return search(createWeight(query), filter, n);
   }
 
   /** Finds the top <code>n</code>
@@ -202,7 +202,7 @@
    * entire index.
    */
   public Explanation explain(Query query, int doc) throws IOException {
-    return explain(createQueryWeight(query), doc);
+    return explain(createWeight(query), doc);
   }
 
   /** The Similarity implementation used by this searcher. */
@@ -215,7 +215,7 @@
   public void setSimilarity(Similarity similarity) {
     this.similarity = similarity;
   }
-  
+
   /** Expert: Return the Similarity implementation used by this Searcher.
    *
    * <p>This defaults to the current value of {@link Similarity#getDefault()}.
@@ -226,15 +226,10 @@
 
   /**
    * creates a weight for <code>query</code>
-   * 
-   * @deprecated use {@link #createQueryWeight(Query)} instead.
+   * @return new weight
    */
   protected Weight createWeight(Query query) throws IOException {
-      return createQueryWeight(query);
-  }
-  
-  protected QueryWeight createQueryWeight(Query query) throws IOException {
-    return query.queryWeight(this);
+    return query.weight(this);
   }
 
   // inherit javadoc
@@ -253,33 +248,16 @@
    * @deprecated use {@link #search(Weight, Filter, Collector)} instead.
    */
   public void search(Weight weight, Filter filter, HitCollector results) throws IOException {
-    search(new QueryWeightWrapper(weight), filter, new HitCollectorWrapper(results));
+    search(weight, filter, new HitCollectorWrapper(results));
   }
-  /** @deprecated delete in 3.0. */
-  public void search(Weight weight, Filter filter, Collector collector)
-      throws IOException {
-    search(new QueryWeightWrapper(weight), filter, collector);
-  }
-  abstract public void search(QueryWeight weight, Filter filter, Collector 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;
-  /** @deprecated use {@link #search(QueryWeight, Filter, int)} instead. */
-  public TopDocs search(Weight weight, Filter filter, int n) throws IOException {
-    return search(new QueryWeightWrapper(weight), filter, n);
-  }
-  abstract public TopDocs search(QueryWeight weight, Filter filter, int n) throws IOException;
+  abstract public TopDocs search(Weight weight, Filter filter, int n) throws IOException;
   abstract public Document doc(int i) throws CorruptIndexException, IOException;
   abstract public Query rewrite(Query query) throws IOException;
-  /** @deprecated use {@link #explain(QueryWeight, int)} instead. */
-  public Explanation explain(Weight weight, int doc) throws IOException {
-    return explain(new QueryWeightWrapper(weight), doc);
-  }
-  abstract public Explanation explain(QueryWeight weight, int doc) throws IOException;
-  /** @deprecated use {@link #search(QueryWeight, Filter, int, Sort)} instead. */
-  public TopFieldDocs search(Weight weight, Filter filter, int n, Sort sort) throws IOException {
-    return search(new QueryWeightWrapper(weight), filter, n, sort);
-  }
-  abstract public TopFieldDocs search(QueryWeight weight, Filter filter, int n, Sort sort) throws IOException;
+  abstract public Explanation explain(Weight weight, int doc) throws IOException;
+  abstract public TopFieldDocs search(Weight weight, Filter filter, int n, Sort sort) throws IOException;
   /* End patch for GCJ bug #15411. */
 }

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/SloppyPhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/SloppyPhraseScorer.java?rev=803339&r1=803338&r2=803339&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/SloppyPhraseScorer.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/SloppyPhraseScorer.java Wed Aug 12 01:22:30 2009
@@ -28,7 +28,7 @@
     private PhrasePositions tmpPos[]; // for flipping repeating pps.
     private boolean checkedRepeats;
 
-    SloppyPhraseScorer(QueryWeight weight, TermPositions[] tps, int[] offsets, Similarity similarity,
+    SloppyPhraseScorer(Weight weight, TermPositions[] tps, int[] offsets, Similarity similarity,
                        int slop, byte[] norms) {
         super(weight, tps, offsets, similarity, norms);
         this.slop = slop;

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/TermQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/TermQuery.java?rev=803339&r1=803338&r2=803339&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/TermQuery.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/TermQuery.java Wed Aug 12 01:22:30 2009
@@ -31,7 +31,7 @@
 public class TermQuery extends Query {
   private Term term;
 
-  private class TermWeight extends QueryWeight {
+  private class TermWeight extends Weight {
     private Similarity similarity;
     private float value;
     private float idf;
@@ -69,15 +69,19 @@
       return new TermScorer(this, termDocs, similarity, reader.norms(term.field()));
     }
 
-    public Explanation explain(IndexReader reader, int doc)
+    public Explanation explain(Searcher searcher, IndexReader reader, int doc)
       throws IOException {
 
       ComplexExplanation result = new ComplexExplanation();
       result.setDescription("weight("+getQuery()+" in "+doc+"), product of:");
 
-      Explanation idfExpl =
-        new Explanation(idf, "idf(docFreq=" + reader.docFreq(term) +
-            ", numDocs=" + reader.numDocs() + ")");
+      Explanation expl;
+      if(searcher == null) {
+        expl = new Explanation(idf, "idf(" + idf + ")");
+      } else {
+        expl = new Explanation(idf, "idf(docFreq=" + searcher.docFreq(term) +
+            ", maxDocs=" + searcher.maxDoc() + ")");
+      }
 
       // explain query weight
       Explanation queryExpl = new Explanation();
@@ -86,13 +90,13 @@
       Explanation boostExpl = new Explanation(getBoost(), "boost");
       if (getBoost() != 1.0f)
         queryExpl.addDetail(boostExpl);
-      queryExpl.addDetail(idfExpl);
+      queryExpl.addDetail(expl);
 
       Explanation queryNormExpl = new Explanation(queryNorm,"queryNorm");
       queryExpl.addDetail(queryNormExpl);
 
       queryExpl.setValue(boostExpl.getValue() *
-                         idfExpl.getValue() *
+                         expl.getValue() *
                          queryNormExpl.getValue());
 
       result.addDetail(queryExpl);
@@ -105,7 +109,7 @@
 
       Explanation tfExpl = scorer(reader, true, false).explain(doc);
       fieldExpl.addDetail(tfExpl);
-      fieldExpl.addDetail(idfExpl);
+      fieldExpl.addDetail(expl);
 
       Explanation fieldNormExpl = new Explanation();
       byte[] fieldNorms = reader.norms(field);
@@ -117,7 +121,7 @@
       
       fieldExpl.setMatch(Boolean.valueOf(tfExpl.isMatch()));
       fieldExpl.setValue(tfExpl.getValue() *
-                         idfExpl.getValue() *
+                         expl.getValue() *
                          fieldNormExpl.getValue());
 
       result.addDetail(fieldExpl);
@@ -141,7 +145,7 @@
   /** Returns the term of this query. */
   public Term getTerm() { return term; }
 
-  public QueryWeight createQueryWeight(Searcher searcher) throws IOException {
+  public Weight createWeight(Searcher searcher) throws IOException {
     return new TermWeight(searcher);
   }
 

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=803339&r1=803338&r2=803339&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 Wed Aug 12 01:22:30 2009
@@ -27,7 +27,7 @@
   
   private static final float[] SIM_NORM_DECODER = Similarity.getNormDecoder();
   
-  private QueryWeight weight;
+  private Weight weight;
   private TermDocs termDocs;
   private byte[] norms;
   private float weightValue;
@@ -53,30 +53,8 @@
    *          computations.
    * @param norms
    *          The field norms of the document fields for the <code>Term</code>.
-   * 
-   * @deprecated use delete in 3.0, kept around for TestTermScorer in tag which
-   *             creates TermScorer directly, and cannot pass in a QueryWeight
-   *             object.
    */
   TermScorer(Weight weight, TermDocs td, Similarity similarity, byte[] norms) {
-    this(new QueryWeightWrapper(weight), td, similarity, norms);
-  }
-
-  /**
-   * Construct a <code>TermScorer</code>.
-   * 
-   * @param weight
-   *          The weight of the <code>Term</code> in the query.
-   * @param td
-   *          An iterator over the documents matching the <code>Term</code>.
-   * @param similarity
-   *          The </code>Similarity</code> implementation to be used for score
-   *          computations.
-   * @param norms
-   *          The field norms of the document fields for the <code>Term</code>.
-   */
-  TermScorer(QueryWeight weight, TermDocs td, Similarity similarity,
-             byte[] norms) {
     super(similarity);
     this.weight = weight;
     this.termDocs = td;

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/Weight.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/Weight.java?rev=803339&r1=803338&r2=803339&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/Weight.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/Weight.java Wed Aug 12 01:22:30 2009
@@ -18,47 +18,106 @@
  */
 
 import java.io.IOException;
+import java.io.Serializable;
 
 import org.apache.lucene.index.IndexReader;
 
-/** Expert: Calculate query weights and build query scorers.
+/**
+ * Expert: Calculate query weights and build query scorers.
  * <p>
- * The purpose of Weight is to make it so that searching does not modify
- * a Query, so that a Query instance can be reused. <br>
- * Searcher dependent state of the query should reside in the Weight. <br>
- * IndexReader dependent state should reside in the Scorer.
+ * The purpose of {@link Weight} is to ensure searching does not
+ * modify a {@link Query}, so that a {@link Query} instance can be reused. <br>
+ * {@link Searcher} dependent state of the query should reside in the
+ * {@link Weight}. <br>
+ * {@link IndexReader} dependent state should reside in the {@link Scorer}.
  * <p>
  * A <code>Weight</code> is used in the following way:
  * <ol>
- * <li>A <code>Weight</code> is constructed by a top-level query,
- *     given a <code>Searcher</code> ({@link Query#createWeight(Searcher)}).
- * <li>The {@link #sumOfSquaredWeights()} method is called
- *     on the <code>Weight</code> to compute
- *     the query normalization factor {@link Similarity#queryNorm(float)}
- *     of the query clauses contained in the query.
- * <li>The query normalization factor is passed to {@link #normalize(float)}.
- *     At this point the weighting is complete.
+ * <li>A <code>Weight</code> is constructed by a top-level query, given a
+ * <code>Searcher</code> ({@link Query#createWeight(Searcher)}).
+ * <li>The {@link #sumOfSquaredWeights()} method is called on the
+ * <code>Weight</code> to compute the query normalization factor
+ * {@link Similarity#queryNorm(float)} of the query clauses contained in the
+ * query.
+ * <li>The query normalization factor is passed to {@link #normalize(float)}. At
+ * this point the weighting is complete.
  * <li>A <code>Scorer</code> is constructed by {@link #scorer(IndexReader)}.
  * </ol>
  * 
- * @deprecated use {@link QueryWeight} instead. 
+ * @since 2.9
  */
-public interface Weight extends java.io.Serializable {
+public abstract class Weight implements Serializable {
+
+  /**
+   * An explanation of the score computation for the named document.
+   * 
+   * Until 3.0, null may be passed in situations where the {@Searcher} is not
+   * available, so impls must only use {@Searcher} to generate optional 
+   * explain info.
+   * 
+   * @param searcher over the index or null
+   * @param reader sub-reader containing the give doc
+   * @param doc
+   * @return an Explanation for the score
+   * @throws IOException
+   */
+  public abstract Explanation explain(Searcher searcher, IndexReader reader, int doc) throws IOException;
+
   /** The query that this concerns. */
-  Query getQuery();
+  public abstract Query getQuery();
 
   /** The weight for this query. */
-  float getValue();
-
-  /** The sum of squared weights of contained query clauses. */
-  float sumOfSquaredWeights() throws IOException;
+  public abstract float getValue();
 
   /** Assigns the query normalization factor to this. */
-  void normalize(float norm);
+  public abstract void normalize(float norm);
+
+  /**
+   * Returns a {@link Scorer} which scores documents in/out-of order according
+   * to <code>scoreDocsInOrder</code>.
+   * <p>
+   * <b>NOTE:</b> even if <code>scoreDocsInOrder</code> is false, it is
+   * recommended to check whether the returned <code>Scorer</code> indeed scores
+   * documents out of order (i.e., call {@link #scoresDocsOutOfOrder()}), as
+   * some <code>Scorer</code> implementations will always return documents
+   * in-order.<br>
+   * <b>NOTE:</b> null can be returned if no documents will be scored by this
+   * query.
+   * 
+   * @param reader
+   *          the {@link IndexReader} for which to return the {@link Scorer}.
+   * @param scoreDocsInOrder
+   *          specifies whether in-order scoring of documents is required. Note
+   *          that if set to false (i.e., out-of-order scoring is required),
+   *          this method can return whatever scoring mode it supports, as every
+   *          in-order scorer is also an out-of-order one. However, an
+   *          out-of-order scorer may not support {@link Scorer#nextDoc()}
+   *          and/or {@link Scorer#advance(int)}, therefore it is recommended to
+   *          request an in-order scorer if use of these methods is required.
+   * @param topScorer
+   *          if true, {@link Scorer#score(Collector)} will be called; if false,
+   *          {@link Scorer#nextDoc()} and/or {@link Scorer#advance(int)} will
+   *          be called.
+   * @return a {@link Scorer} which scores documents in/out-of order.
+   * @throws IOException
+   */
+  public abstract Scorer scorer(IndexReader reader, boolean scoreDocsInOrder,
+      boolean topScorer) throws IOException;
+  
+  /** The sum of squared weights of contained query clauses. */
+  public abstract float sumOfSquaredWeights() throws IOException;
 
-  /** Constructs a scorer for this. */
-  Scorer scorer(IndexReader reader) throws IOException;
+  /**
+   * Returns true iff this implementation scores docs only out of order. This
+   * method is used in conjunction with {@link Collector}'s
+   * {@link Collector#acceptsDocsOutOfOrder() acceptsDocsOutOfOrder} and
+   * {@link #scorer(org.apache.lucene.index.IndexReader, boolean, boolean)} to
+   * create a matching {@link Scorer} instance for a given {@link Collector}, or
+   * vice versa.
+   * <p>
+   * <b>NOTE:</b> the default implementation returns <code>false</code>, i.e.
+   * the <code>Scorer</code> scores documents in-order.
+   */
+  public boolean scoresDocsOutOfOrder() { return false; }
 
-  /** An explanation of the score computation for the named document. */
-  Explanation explain(IndexReader reader, int doc) throws IOException;
 }

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/function/CustomScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/function/CustomScoreQuery.java?rev=803339&r1=803338&r2=803339&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/function/CustomScoreQuery.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/function/CustomScoreQuery.java Wed Aug 12 01:22:30 2009
@@ -24,7 +24,7 @@
 import org.apache.lucene.search.ComplexExplanation;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.search.QueryWeight;
+import org.apache.lucene.search.Weight;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Searcher;
 import org.apache.lucene.search.Similarity;
@@ -271,18 +271,18 @@
 
   //=========================== W E I G H T ============================
   
-  private class CustomWeight extends QueryWeight {
+  private class CustomWeight extends Weight {
     Similarity similarity;
-    QueryWeight subQueryWeight;
-    QueryWeight[] valSrcWeights;
+    Weight subQueryWeight;
+    Weight[] valSrcWeights;
     boolean qStrict;
 
     public CustomWeight(Searcher searcher) throws IOException {
       this.similarity = getSimilarity(searcher);
-      this.subQueryWeight = subQuery.queryWeight(searcher);
-      this.valSrcWeights = new QueryWeight[valSrcQueries.length];
+      this.subQueryWeight = subQuery.weight(searcher);
+      this.valSrcWeights = new Weight[valSrcQueries.length];
       for(int i = 0; i < valSrcQueries.length; i++) {
-        this.valSrcWeights[i] = valSrcQueries[i].createQueryWeight(searcher);
+        this.valSrcWeights[i] = valSrcQueries[i].createWeight(searcher);
       }
       this.qStrict = strict;
     }
@@ -336,16 +336,39 @@
       }
       Scorer[] valSrcScorers = new Scorer[valSrcWeights.length];
       for(int i = 0; i < valSrcScorers.length; i++) {
-         valSrcScorers[i] = valSrcWeights[i].scorer(reader, true, false);
+         valSrcScorers[i] = valSrcWeights[i].scorer(reader, true, topScorer);
       }
       return new CustomScorer(similarity, reader, this, subQueryScorer, valSrcScorers);
     }
 
-    public Explanation explain(IndexReader reader, int doc) throws IOException {
-      Scorer scorer = scorer(reader, true, false);
-      return scorer == null ? new Explanation(0.0f, "no matching docs") : scorer.explain(doc);
+    public Explanation explain(Searcher searcher, IndexReader reader, int doc) throws IOException {
+      Explanation explain = doExplain(searcher, reader, doc);
+      return explain == null ? new Explanation(0.0f, "no matching docs") : doExplain(searcher, reader, doc);
     }
     
+    private Explanation doExplain(Searcher searcher, IndexReader reader, int doc) throws IOException {
+      Scorer[] valSrcScorers = new Scorer[valSrcWeights.length];
+      for(int i = 0; i < valSrcScorers.length; i++) {
+         valSrcScorers[i] = valSrcWeights[i].scorer(reader, true, false);
+      }
+      Explanation subQueryExpl = subQueryWeight.explain(searcher, reader, doc);
+      if (!subQueryExpl.isMatch()) {
+        return subQueryExpl;
+      }
+      // match
+      Explanation[] valSrcExpls = new Explanation[valSrcScorers.length];
+      for(int i = 0; i < valSrcScorers.length; i++) {
+        valSrcExpls[i] = valSrcScorers[i].explain(doc);
+      }
+      Explanation customExp = customExplain(doc,subQueryExpl,valSrcExpls);
+      float sc = getValue() * customExp.getValue();
+      Explanation res = new ComplexExplanation(
+        true, sc, CustomScoreQuery.this.toString() + ", product of:");
+      res.addDetail(customExp);
+      res.addDetail(new Explanation(getValue(), "queryBoost")); // actually using the q boost as q weight (== weight value)
+      return res;
+    }
+
     public boolean scoresDocsOutOfOrder() {
       return false;
     }
@@ -425,9 +448,10 @@
       return doc;
     }
     
+    // TODO: remove in 3.0
     /*(non-Javadoc) @see org.apache.lucene.search.Scorer#explain(int) */
     public Explanation explain(int doc) throws IOException {
-      Explanation subQueryExpl = weight.subQueryWeight.explain(reader,doc);
+      Explanation subQueryExpl = weight.subQueryWeight.explain(null, reader,doc); // nocommit: needs resolution
       if (!subQueryExpl.isMatch()) {
         return subQueryExpl;
       }
@@ -446,7 +470,7 @@
     }
   }
 
-  public QueryWeight createQueryWeight(Searcher searcher) throws IOException {
+  public Weight createWeight(Searcher searcher) throws IOException {
     return new CustomWeight(searcher);
   }
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/function/ValueSourceQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/function/ValueSourceQuery.java?rev=803339&r1=803338&r2=803339&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/function/ValueSourceQuery.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/function/ValueSourceQuery.java Wed Aug 12 01:22:30 2009
@@ -62,7 +62,7 @@
     // no terms involved here
   }
 
-  private class ValueSourceWeight extends QueryWeight {
+  class ValueSourceWeight extends Weight {
     Similarity similarity;
     float queryNorm;
     float queryWeight;
@@ -98,8 +98,8 @@
     }
 
     /*(non-Javadoc) @see org.apache.lucene.search.Weight#explain(org.apache.lucene.index.IndexReader, int) */
-    public Explanation explain(IndexReader reader, int doc) throws IOException {
-      return scorer(reader, true, false).explain(doc);
+    public Explanation explain(Searcher searcher, IndexReader reader, int doc) throws IOException {
+      return new ValueSourceScorer(similarity, reader, this).explain(doc);
     }
   }
 
@@ -172,7 +172,7 @@
     }
   }
 
-  public QueryWeight createQueryWeight(Searcher searcher) {
+  public Weight createWeight(Searcher searcher) {
     return new ValueSourceQuery.ValueSourceWeight(searcher);
   }
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/payloads/BoostingFunctionTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/payloads/BoostingFunctionTermQuery.java?rev=803339&r1=803338&r2=803339&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/payloads/BoostingFunctionTermQuery.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/payloads/BoostingFunctionTermQuery.java Wed Aug 12 01:22:30 2009
@@ -21,7 +21,7 @@
 import org.apache.lucene.index.TermPositions;
 import org.apache.lucene.search.Searcher;
 import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.QueryWeight;
+import org.apache.lucene.search.Weight;
 import org.apache.lucene.search.Similarity;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.ComplexExplanation;
@@ -54,7 +54,7 @@
 
   
 
-  public QueryWeight createQueryWeight(Searcher searcher) throws IOException {
+  public Weight createWeight(Searcher searcher) throws IOException {
     return new BoostingFunctionTermWeight(this, searcher);
   }
 
@@ -76,7 +76,7 @@
       protected float payloadScore;
       protected int payloadsSeen;
 
-      public BoostingFunctionSpanScorer(TermSpans spans, QueryWeight weight, Similarity similarity,
+      public BoostingFunctionSpanScorer(TermSpans spans, Weight weight, Similarity similarity,
                                    byte[] norms) throws IOException {
         super(spans, weight, similarity, norms);
         positions = spans.getPositions();

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/payloads/BoostingNearQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/payloads/BoostingNearQuery.java?rev=803339&r1=803338&r2=803339&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/payloads/BoostingNearQuery.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/payloads/BoostingNearQuery.java Wed Aug 12 01:22:30 2009
@@ -18,7 +18,6 @@
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.search.Explanation;
-import org.apache.lucene.search.QueryWeight;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Searcher;
 import org.apache.lucene.search.Similarity;
@@ -63,7 +62,7 @@
   }
 
 
-  public QueryWeight createQueryWeight(Searcher searcher) throws IOException {
+  public Weight createWeight(Searcher searcher) throws IOException {
     return new BoostingSpanWeight(this, searcher);
   }
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/payloads/BoostingTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/payloads/BoostingTermQuery.java?rev=803339&r1=803338&r2=803339&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/payloads/BoostingTermQuery.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/payloads/BoostingTermQuery.java Wed Aug 12 01:22:30 2009
@@ -1,16 +1,14 @@
 package org.apache.lucene.search.payloads;
 
+import java.io.IOException;
+
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.index.TermPositions;
-import org.apache.lucene.search.*;
-import org.apache.lucene.search.spans.SpanScorer;
-import org.apache.lucene.search.spans.SpanTermQuery;
-import org.apache.lucene.search.spans.SpanWeight;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Searcher;
+import org.apache.lucene.search.Weight;
 import org.apache.lucene.search.spans.TermSpans;
 
-import java.io.IOException;
-
 /**
  * Copyright 2004 The Apache Software Foundation
  * <p/>
@@ -51,7 +49,7 @@
     super(term, new AveragePayloadFunction(), includeSpanScore);
   }
 
-  public QueryWeight createQueryWeight(Searcher searcher) throws IOException {
+  public Weight createWeight(Searcher searcher) throws IOException {
     return new BoostingTermWeight(this, searcher);
   }
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java?rev=803339&r1=803338&r2=803339&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java Wed Aug 12 01:22:30 2009
@@ -23,7 +23,7 @@
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.search.QueryWeight;
+import org.apache.lucene.search.Weight;
 import org.apache.lucene.search.Searcher;
 import org.apache.lucene.search.Similarity;
 import org.apache.lucene.util.ToStringUtils;
@@ -103,8 +103,8 @@
     maskedQuery.extractTerms(terms);
   }  
 
-  public QueryWeight createQueryWeight(Searcher searcher) throws IOException {
-    return maskedQuery.createQueryWeight(searcher);
+  public Weight createWeight(Searcher searcher) throws IOException {
+    return maskedQuery.createWeight(searcher);
   }
 
   public Similarity getSimilarity(Searcher searcher) {

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanQuery.java?rev=803339&r1=803338&r2=803339&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanQuery.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanQuery.java Wed Aug 12 01:22:30 2009
@@ -22,7 +22,6 @@
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.search.QueryWeight;
 import org.apache.lucene.search.Searcher;
 import org.apache.lucene.search.Weight;
 
@@ -40,13 +39,8 @@
    * @see Query#extractTerms(Set)
    */
   public abstract Collection getTerms();
-
-  /** @deprecated delete in 3.0. */
-  protected Weight createWeight(Searcher searcher) throws IOException {
-    return createQueryWeight(searcher);
-  }
   
-  public QueryWeight createQueryWeight(Searcher searcher) throws IOException {
+  public Weight createWeight(Searcher searcher) throws IOException {
     return new SpanWeight(this, searcher);
   }
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanScorer.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanScorer.java?rev=803339&r1=803338&r2=803339&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanScorer.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanScorer.java Wed Aug 12 01:22:30 2009
@@ -20,18 +20,16 @@
 import java.io.IOException;
 
 import org.apache.lucene.search.Explanation;
-import org.apache.lucene.search.QueryWeight;
-import org.apache.lucene.search.QueryWeightWrapper;
+import org.apache.lucene.search.Weight;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Similarity;
-import org.apache.lucene.search.Weight;
 
 /**
  * Public for extension only.
  */
 public class SpanScorer extends Scorer {
   protected Spans spans;
-  protected QueryWeight weight;
+  protected Weight weight;
   protected byte[] norms;
   protected float value;
 
@@ -42,13 +40,7 @@
   protected int doc;
   protected float freq;
 
-  /** @deprecated use {@link #SpanScorer(Spans, QueryWeight, Similarity, byte[])} instead.*/
   protected SpanScorer(Spans spans, Weight weight, Similarity similarity, byte[] norms)
-    throws IOException {
-    this(spans, new QueryWeightWrapper(weight), similarity, norms);
-  }
-  
-  protected SpanScorer(Spans spans, QueryWeight weight, Similarity similarity, byte[] norms)
   throws IOException {
     super(similarity);
     this.spans = spans;

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanWeight.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanWeight.java?rev=803339&r1=803338&r2=803339&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanWeight.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanWeight.java Wed Aug 12 01:22:30 2009
@@ -29,7 +29,7 @@
 /**
  * Expert-only.  Public for use by other weight implementations
  */
-public class SpanWeight extends QueryWeight {
+public class SpanWeight extends Weight {
   protected Similarity similarity;
   protected float value;
   protected float idf;
@@ -68,7 +68,7 @@
         .norms(query.getField()));
   }
 
-  public Explanation explain(IndexReader reader, int doc)
+  public Explanation explain(Searcher searcher, IndexReader reader, int doc)
     throws IOException {
 
     ComplexExplanation result = new ComplexExplanation();

Added: lucene/java/trunk/src/java/org/apache/lucene/util/ReaderUtil.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/util/ReaderUtil.java?rev=803339&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/util/ReaderUtil.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/util/ReaderUtil.java Wed Aug 12 01:22:30 2009
@@ -0,0 +1,107 @@
+package org.apache.lucene.util;
+
+/**
+ * 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.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.index.IndexReader;
+
+public class ReaderUtil {
+
+  /**
+   * Gathers sub-readers from reader into a List.
+   * 
+   * @param allSubReaders
+   * @param reader
+   */
+  public static void gatherSubReaders(List allSubReaders, IndexReader reader) {
+    IndexReader[] subReaders = reader.getSequentialSubReaders();
+    if (subReaders == null) {
+      // Add the reader itself, and do not recurse
+      allSubReaders.add(reader);
+    } else {
+      for (int i = 0; i < subReaders.length; i++) {
+        gatherSubReaders(allSubReaders, subReaders[i]);
+      }
+    }
+  }
+
+  /**
+   * Returns sub IndexReader that contains the given document id.
+   *    
+   * @param doc
+   * @param reader
+   * @return
+   */
+  public static IndexReader subReader(int doc, IndexReader reader) {
+    List subReadersList = new ArrayList();
+    ReaderUtil.gatherSubReaders(subReadersList, reader);
+    IndexReader[] subReaders = (IndexReader[]) subReadersList
+        .toArray(new IndexReader[subReadersList.size()]);
+    int[] docStarts = new int[subReaders.length];
+    int maxDoc = 0;
+    for (int i = 0; i < subReaders.length; i++) {
+      docStarts[i] = maxDoc;
+      maxDoc += subReaders[i].maxDoc();
+    }
+    return subReaders[ReaderUtil.subIndex(doc, docStarts)];
+  }
+  
+  /**
+   * Returns sub-reader subIndex from reader.
+   * 
+   * @param reader
+   * @param subIndex
+   * @return
+   */
+  public static IndexReader subReader(IndexReader reader, int subIndex) {
+    List subReadersList = new ArrayList();
+    ReaderUtil.gatherSubReaders(subReadersList, reader);
+    IndexReader[] subReaders = (IndexReader[]) subReadersList
+        .toArray(new IndexReader[subReadersList.size()]);
+    return subReaders[subIndex];
+  }
+
+
+  /**
+   * Returns index of the searcher/reader for document <code>n</code> in the
+   * array used to construct this searcher/reader.
+   */
+  public static int subIndex(int n, int[] docStarts) { // find
+    // searcher/reader for doc n:
+    int size = docStarts.length;
+    int lo = 0; // search starts array
+    int hi = size - 1; // for first element less than n, return its index
+    while (hi >= lo) {
+      int mid = (lo + hi) >>> 1;
+      int midValue = docStarts[mid];
+      if (n < midValue)
+        hi = mid - 1;
+      else if (n > midValue)
+        lo = mid + 1;
+      else { // found a match
+        while (mid + 1 < size && docStarts[mid + 1] == midValue) {
+          mid++; // scan to last match
+        }
+        return mid;
+      }
+    }
+    return hi;
+  }
+}