You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2011/02/09 10:36:03 UTC

svn commit: r1068809 [13/36] - in /lucene/dev/branches/docvalues: ./ dev-tools/eclipse/ dev-tools/idea/.idea/ dev-tools/idea/.idea/copyright/ dev-tools/idea/lucene/ dev-tools/idea/lucene/contrib/ant/ dev-tools/idea/lucene/contrib/queryparser/ dev-tools...

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Similarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Similarity.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Similarity.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Similarity.java Wed Feb  9 09:35:27 2011
@@ -362,7 +362,7 @@ import org.apache.lucene.util.SmallFloat
  *      Typically, a document that contains more of the query's terms will receive a higher score
  *      than another document with fewer query terms.
  *      This is a search time factor computed in
- *      {@link #coord(int, int) coord(q,d)}
+ *      {@link SimilarityProvider#coord(int, int) coord(q,d)}
  *      by the Similarity in effect at search time.
  *      <br>&nbsp;<br>
  *    </li>
@@ -462,12 +462,14 @@ import org.apache.lucene.util.SmallFloat
  *        {@link org.apache.lucene.document.Fieldable#setBoost(float) field.setBoost()}
  *        before adding the field to a document.
  *        </li>
- *        <li>{@link #lengthNorm(String, int) <b>lengthNorm</b>(field)} - computed
+ *        <li><b>lengthNorm</b> - computed
  *        when the document is added to the index in accordance with the number of tokens
  *        of this field in the document, so that shorter fields contribute more to the score.
  *        LengthNorm is computed by the Similarity class in effect at indexing.
  *        </li>
  *      </ul>
+ *      The {@link #computeNorm} method is responsible for
+ *      combining all of these factors into a single float.
  *
  *      <p>
  *      When a document is added to the index, all the above factors are multiplied.
@@ -480,7 +482,7 @@ import org.apache.lucene.util.SmallFloat
  *            norm(t,d) &nbsp; = &nbsp;
  *            {@link org.apache.lucene.document.Document#getBoost() doc.getBoost()}
  *            &nbsp;&middot;&nbsp;
- *            {@link #lengthNorm(String, int) lengthNorm(field)}
+ *            lengthNorm
  *            &nbsp;&middot;&nbsp;
  *          </td>
  *          <td valign="bottom" align="center" rowspan="1">
@@ -520,40 +522,13 @@ import org.apache.lucene.util.SmallFloat
  *    </li>
  * </ol>
  *
- * @see #setDefault(Similarity)
- * @see org.apache.lucene.index.IndexWriter#setSimilarity(Similarity)
- * @see Searcher#setSimilarity(Similarity)
+ * @see org.apache.lucene.index.IndexWriterConfig#setSimilarityProvider(SimilarityProvider)
+ * @see IndexSearcher#setSimilarityProvider(SimilarityProvider)
  */
 public abstract class Similarity implements Serializable {
   
-  /**
-   * The Similarity implementation used by default.
-   **/
-  private static Similarity defaultImpl = new DefaultSimilarity();
   public static final int NO_DOC_ID_PROVIDED = -1;
 
-  /** Set the default Similarity implementation used by indexing and search
-   * code.
-   *
-   * @see Searcher#setSimilarity(Similarity)
-   * @see org.apache.lucene.index.IndexWriter#setSimilarity(Similarity)
-   */
-  public static void setDefault(Similarity similarity) {
-    Similarity.defaultImpl = similarity;
-  }
-
-  /** Return the default Similarity implementation used by indexing and search
-   * code.
-   *
-   * <p>This is initially an instance of {@link DefaultSimilarity}.
-   *
-   * @see Searcher#setSimilarity(Similarity)
-   * @see org.apache.lucene.index.IndexWriter#setSimilarity(Similarity)
-   */
-  public static Similarity getDefault() {
-    return Similarity.defaultImpl;
-  }
-
   /** Cache of decoded bytes. */
   private static final float[] NORM_TABLE = new float[256];
 
@@ -570,12 +545,23 @@ public abstract class Similarity impleme
   }
 
   /**
-   * Compute the normalization value for a field, given the accumulated
+   * Computes the normalization value for a field, given the accumulated
    * state of term processing for this field (see {@link FieldInvertState}).
    * 
    * <p>Implementations should calculate a float value based on the field
    * state and then return that value.
    *
+   * <p>Matches in longer fields are less precise, so implementations of this
+   * method usually return smaller values when <code>state.getLength()</code> is large,
+   * and larger values when <code>state.getLength()</code> is small.
+   * 
+   * <p>Note that the return values are computed under 
+   * {@link org.apache.lucene.index.IndexWriter#addDocument(org.apache.lucene.document.Document)} 
+   * and then stored using
+   * {@link #encodeNormValue(float)}.  
+   * Thus they have limited precision, and documents
+   * must be re-indexed if this method is altered.
+   *
    * <p>For backward compatibility this method by default calls
    * {@link #lengthNorm(String, int)} passing
    * {@link FieldInvertState#getLength()} as the second argument, and
@@ -587,9 +573,7 @@ public abstract class Similarity impleme
    * @param state current processing state for this field
    * @return the calculated float norm
    */
-  public float computeNorm(String field, FieldInvertState state) {
-    return (state.getBoost() * lengthNorm(field, state.getLength()));
-  }
+  public abstract float computeNorm(String field, FieldInvertState state);
   
   /** Computes the normalization value for a field given the total number of
    * terms contained in a field.  These values, together with field boosts, are
@@ -613,23 +597,13 @@ public abstract class Similarity impleme
    * @return a normalization factor for hits on this field of this document
    *
    * @see org.apache.lucene.document.Field#setBoost(float)
-   */
-  public abstract float lengthNorm(String fieldName, int numTokens);
-
-  /** Computes the normalization value for a query given the sum of the squared
-   * weights of each of the query terms.  This value is multiplied into the
-   * weight of each query term. While the classic query normalization factor is
-   * computed as 1/sqrt(sumOfSquaredWeights), other implementations might
-   * completely ignore sumOfSquaredWeights (ie return 1).
-   *
-   * <p>This does not affect ranking, but the default implementation does make scores
-   * from different queries more comparable than they would be by eliminating the
-   * magnitude of the Query vector as a factor in the score.
    *
-   * @param sumOfSquaredWeights the sum of the squares of query term weights
-   * @return a normalization factor for query weights
+   * @deprecated Please override computeNorm instead
    */
-  public abstract float queryNorm(float sumOfSquaredWeights);
+  @Deprecated
+  public final float lengthNorm(String fieldName, int numTokens) {
+    throw new UnsupportedOperationException("please use computeNorm instead");
+  }
 
   /** Encodes a normalization factor for storage in an index.
    *
@@ -641,7 +615,6 @@ public abstract class Similarity impleme
    * are rounded down to the largest representable value.  Positive values too
    * small to represent are rounded up to the smallest positive representable
    * value.
-   *
    * @see org.apache.lucene.document.Field#setBoost(float)
    * @see org.apache.lucene.util.SmallFloat
    */
@@ -709,11 +682,11 @@ public abstract class Similarity impleme
    * idf(docFreq, searcher.maxDoc());
    * </pre>
    * 
-   * Note that {@link Searcher#maxDoc()} is used instead of
+   * Note that {@link IndexSearcher#maxDoc()} is used instead of
    * {@link org.apache.lucene.index.IndexReader#numDocs() IndexReader#numDocs()} because also 
-   * {@link Searcher#docFreq(Term)} is used, and when the latter 
-   * is inaccurate, so is {@link Searcher#maxDoc()}, and in the same direction.
-   * In addition, {@link Searcher#maxDoc()} is more efficient to compute
+   * {@link IndexSearcher#docFreq(Term)} is used, and when the latter 
+   * is inaccurate, so is {@link IndexSearcher#maxDoc()}, and in the same direction.
+   * In addition, {@link IndexSearcher#maxDoc()} is more efficient to compute
    *   
    * @param term the term in question
    * @param searcher the document collection being searched
@@ -736,16 +709,16 @@ public abstract class Similarity impleme
         public float getIdf() {
           return idf;
         }};
-   }
+  }
 
   /**
    * This method forwards to {@link
-   * #idfExplain(Term,Searcher,int)} by passing
+   * #idfExplain(Term,IndexSearcher,int)} by passing
    * <code>searcher.docFreq(term)</code> as the docFreq.
    */
   public IDFExplanation idfExplain(final Term term, final IndexSearcher searcher) throws IOException {
     return idfExplain(term, searcher, searcher.docFreq(term));
-   }
+  }
 
   /**
    * Computes a score factor for a phrase.
@@ -801,20 +774,6 @@ public abstract class Similarity impleme
    */
   public abstract float idf(int docFreq, int numDocs);
 
-  /** Computes a score factor based on the fraction of all query terms that a
-   * document contains.  This value is multiplied into scores.
-   *
-   * <p>The presence of a large portion of the query terms indicates a better
-   * match with the query, so implementations of this method usually return
-   * larger values when the ratio between these parameters is large and smaller
-   * values when the ratio between them is small.
-   *
-   * @param overlap the number of query terms matched in the document
-   * @param maxOverlap the total number of terms in the query
-   * @return a score factor based on term overlap with the query
-   */
-  public abstract float coord(int overlap, int maxOverlap);
-
   /**
    * Calculate a scoring factor based on the data in the payload.  Overriding implementations
    * are responsible for interpreting what is in the payload.  Lucene makes no assumptions about

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/SloppyPhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/SloppyPhraseScorer.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/SloppyPhraseScorer.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/SloppyPhraseScorer.java Wed Feb  9 09:35:27 2011
@@ -78,7 +78,7 @@ final class SloppyPhraseScorer extends P
 
             int matchLength = end - start;
             if (matchLength <= slop)
-                freq += getSimilarity().sloppyFreq(matchLength); // score match
+                freq += similarity.sloppyFreq(matchLength); // score match
 
             if (pp.position > end)
                 end = pp.position;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Sort.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Sort.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Sort.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Sort.java Wed Feb  9 09:35:27 2011
@@ -103,7 +103,7 @@ implements Serializable {
   /**
    * Represents sorting by computed relevance. Using this sort criteria returns
    * the same results as calling
-   * {@link Searcher#search(Query,int) Searcher#search()}without a sort criteria,
+   * {@link IndexSearcher#search(Query,int) IndexSearcher#search()}without a sort criteria,
    * only with slightly more overhead.
    */
   public static final Sort RELEVANCE = new Sort();
@@ -116,7 +116,7 @@ implements Serializable {
 
   /**
    * Sorts by computed relevance. This is the same sort criteria as calling
-   * {@link Searcher#search(Query,int) Searcher#search()}without a sort criteria,
+   * {@link IndexSearcher#search(Query,int) IndexSearcher#search()}without a sort criteria,
    * only with slightly more overhead.
    */
   public Sort() {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/SortField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/SortField.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/SortField.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/SortField.java Wed Feb  9 09:35:27 2011
@@ -42,7 +42,7 @@ import org.apache.lucene.util.BytesRef;
 public class SortField
 implements Serializable {
 
-  /** Sort by document score (relevancy).  Sort values are Float and higher
+  /** Sort by document score (relevance).  Sort values are Float and higher
    * values are at the front. */
   public static final int SCORE = 0;
 
@@ -93,7 +93,7 @@ implements Serializable {
   /** Sort use byte[] index values. */
   public static final int BYTES = 12;
   
-  /** Represents sorting by document score (relevancy). */
+  /** Represents sorting by document score (relevance). */
   public static final SortField FIELD_SCORE = new SortField (null, SCORE);
 
   /** Represents sorting by document number (index order). */

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/SpanFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/SpanFilter.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/SpanFilter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/SpanFilter.java Wed Feb  9 09:35:27 2011
@@ -15,7 +15,7 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 
 import java.io.IOException;
 
@@ -30,9 +30,9 @@ import java.io.IOException;
 public abstract class SpanFilter extends Filter{
   /** Returns a SpanFilterResult with true for documents which should be permitted in
     search results, and false for those that should not and Spans for where the true docs match.
-   * @param reader The {@link org.apache.lucene.index.IndexReader} to load position and DocIdSet information from
+   * @param context The {@link AtomicReaderContext} to load position and DocIdSet information from
    * @return A {@link SpanFilterResult}
    * @throws java.io.IOException if there was an issue accessing the necessary information
    * */
-  public abstract SpanFilterResult bitSpans(IndexReader reader) throws IOException;
+  public abstract SpanFilterResult bitSpans(AtomicReaderContext context) throws IOException;
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/SpanQueryFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/SpanQueryFilter.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/SpanQueryFilter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/SpanQueryFilter.java Wed Feb  9 09:35:27 2011
@@ -16,7 +16,7 @@ package org.apache.lucene.search;
  */
 
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.search.spans.SpanQuery;
 import org.apache.lucene.search.spans.Spans;
 import org.apache.lucene.util.OpenBitSet;
@@ -52,16 +52,16 @@ public class SpanQueryFilter extends Spa
   }
 
   @Override
-  public DocIdSet getDocIdSet(IndexReader reader) throws IOException {
-    SpanFilterResult result = bitSpans(reader);
+  public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
+    SpanFilterResult result = bitSpans(context);
     return result.getDocIdSet();
   }
 
   @Override
-  public SpanFilterResult bitSpans(IndexReader reader) throws IOException {
+  public SpanFilterResult bitSpans(AtomicReaderContext context) throws IOException {
 
-    final OpenBitSet bits = new OpenBitSet(reader.maxDoc());
-    Spans spans = query.getSpans(reader);
+    final OpenBitSet bits = new OpenBitSet(context.reader.maxDoc());
+    Spans spans = query.getSpans(context);
     List<SpanFilterResult.PositionInfo> tmp = new ArrayList<SpanFilterResult.PositionInfo>(20);
     int currentDoc = -1;
     SpanFilterResult.PositionInfo currentInfo = null;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TermCollectingRewrite.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TermCollectingRewrite.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TermCollectingRewrite.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TermCollectingRewrite.java Wed Feb  9 09:35:27 2011
@@ -18,8 +18,6 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
 import java.util.Comparator;
 
 import org.apache.lucene.index.Fields;
@@ -27,25 +25,33 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.IndexReader.ReaderContext;
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.PerReaderTermState;
 import org.apache.lucene.util.ReaderUtil;
 
 abstract class TermCollectingRewrite<Q extends Query> extends MultiTermQuery.RewriteMethod {
   
+  
   /** Return a suitable top-level Query for holding all expanded terms. */
   protected abstract Q getTopLevelQuery() throws IOException;
   
   /** Add a MultiTermQuery term to the top-level query */
-  protected abstract void addClause(Q topLevel, Term term, int docCount, float boost) throws IOException;
+  protected final void addClause(Q topLevel, Term term, int docCount, float boost) throws IOException {
+    addClause(topLevel, term, docCount, boost, null);
+  }
+  
+  protected abstract void addClause(Q topLevel, Term term, int docCount, float boost, PerReaderTermState states) throws IOException;
+
   
   protected final void collectTerms(IndexReader reader, MultiTermQuery query, TermCollector collector) throws IOException {
-    final List<IndexReader> subReaders = new ArrayList<IndexReader>();
-    ReaderUtil.gatherSubReaders(subReaders, reader);
+    ReaderContext topReaderContext = reader.getTopReaderContext();
     Comparator<BytesRef> lastTermComp = null;
-    
-    for (IndexReader r : subReaders) {
-      final Fields fields = r.fields();
+    final AtomicReaderContext[] leaves = ReaderUtil.leaves(topReaderContext);
+    for (AtomicReaderContext context : leaves) {
+      final Fields fields = context.reader.fields();
       if (fields == null) {
         // reader has no fields
         continue;
@@ -68,11 +74,10 @@ abstract class TermCollectingRewrite<Q e
       if (lastTermComp != null && newTermComp != null && newTermComp != lastTermComp)
         throw new RuntimeException("term comparator should not change between segments: "+lastTermComp+" != "+newTermComp);
       lastTermComp = newTermComp;
-      
+      collector.setReaderContext(topReaderContext, context);
       collector.setNextEnum(termsEnum);
       BytesRef bytes;
       while ((bytes = termsEnum.next()) != null) {
-        termsEnum.cacheCurrentTerm();
         if (!collector.collect(bytes))
           return; // interrupt whole term collection, so also don't iterate other subReaders
       }
@@ -80,6 +85,14 @@ abstract class TermCollectingRewrite<Q e
   }
   
   protected static abstract class TermCollector {
+    
+    protected AtomicReaderContext readerContext;
+    protected ReaderContext topReaderContext;
+
+    public void setReaderContext(ReaderContext topReaderContext, AtomicReaderContext readerContext) {
+      this.readerContext = readerContext;
+      this.topReaderContext = topReaderContext;
+    }
     /** attributes used for communication with the enum */
     public final AttributeSource attributes = new AttributeSource();
   

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TermQuery.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TermQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TermQuery.java Wed Feb  9 09:35:27 2011
@@ -21,9 +21,16 @@ import java.io.IOException;
 import java.util.Set;
 
 import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.Term;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.TermState;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.IndexReader.ReaderContext;
+import org.apache.lucene.index.Term;
 import org.apache.lucene.search.Explanation.IDFExplanation;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.PerReaderTermState;
+import org.apache.lucene.util.ReaderUtil;
 import org.apache.lucene.util.ToStringUtils;
 
 /** A Query that matches documents containing a term.
@@ -31,19 +38,23 @@ import org.apache.lucene.util.ToStringUt
   */
 public class TermQuery extends Query {
   private final Term term;
-  private final int docFreq;
+  private int docFreq;
+  private transient PerReaderTermState perReaderTermState;
 
   private class TermWeight extends Weight {
     private final Similarity similarity;
     private float value;
-    private float idf;
+    private final float idf;
     private float queryNorm;
     private float queryWeight;
-    private IDFExplanation idfExp;
+    private final IDFExplanation idfExp;
+    private transient PerReaderTermState termStates;
 
-    public TermWeight(IndexSearcher searcher)
+    public TermWeight(IndexSearcher searcher, PerReaderTermState termStates, int docFreq)
       throws IOException {
-      this.similarity = getSimilarity(searcher);
+      assert termStates != null : "PerReaderTermState must not be null";
+      this.termStates = termStates;
+      this.similarity = searcher.getSimilarityProvider().get(term.field());
       if (docFreq != -1) {
         idfExp = similarity.idfExplain(term, searcher, docFreq);
       } else {
@@ -75,21 +86,31 @@ public class TermQuery extends Query {
     }
 
     @Override
-    public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
-      DocsEnum docs = reader.termDocsEnum(reader.getDeletedDocs(),
-                                          term.field(),
-                                          term.bytes());
-
-      if (docs == null) {
+    public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
+      final String field = term.field();
+      final IndexReader reader = context.reader;
+      assert termStates.topReaderContext == ReaderUtil.getTopLevelContext(context) : "The top-reader used to create Weight is not the same as the current reader's top-reader";
+      final TermState state = termStates
+          .get(context.ord);
+      if (state == null) { // term is not present in that reader
+        assert termNotInReader(reader, field, term.bytes()) : "no termstate found but term exists in reader";
         return null;
       }
-
-      return new TermScorer(this, docs, similarity, reader.norms(term.field()));
+      final DocsEnum docs = reader.termDocsEnum(reader.getDeletedDocs(), field, term.bytes(), state);
+      assert docs != null;
+      return new TermScorer(this, docs, similarity, context.reader.norms(field));
+    }
+    
+    private boolean termNotInReader(IndexReader reader, String field, BytesRef bytes) throws IOException {
+      // only called from assert
+      final Terms terms = reader.terms(field);
+      return terms == null || terms.docFreq(bytes) == 0;
     }
-
+    
     @Override
-    public Explanation explain(IndexReader reader, int doc)
+    public Explanation explain(AtomicReaderContext context, int doc)
       throws IOException {
+      final IndexReader reader = context.reader;
 
       ComplexExplanation result = new ComplexExplanation();
       result.setDescription("weight("+getQuery()+" in "+doc+"), product of:");
@@ -138,7 +159,7 @@ public class TermQuery extends Query {
       fieldExpl.addDetail(expl);
 
       Explanation fieldNormExpl = new Explanation();
-      byte[] fieldNorms = reader.norms(field);
+      final byte[] fieldNorms = reader.norms(field);
       float fieldNorm =
         fieldNorms!=null ? similarity.decodeNormValue(fieldNorms[doc]) : 1.0f;
       fieldNormExpl.setValue(fieldNorm);
@@ -174,6 +195,17 @@ public class TermQuery extends Query {
   public TermQuery(Term t, int docFreq) {
     term = t;
     this.docFreq = docFreq;
+    perReaderTermState = null;
+  }
+  
+  /** Expert: constructs a TermQuery that will use the
+   *  provided docFreq instead of looking up the docFreq
+   *  against the searcher. */
+  public TermQuery(Term t, PerReaderTermState states) {
+    assert states != null;
+    term = t;
+    docFreq = states.docFreq();
+    perReaderTermState = states;
   }
 
   /** Returns the term of this query. */
@@ -181,7 +213,21 @@ public class TermQuery extends Query {
 
   @Override
   public Weight createWeight(IndexSearcher searcher) throws IOException {
-    return new TermWeight(searcher);
+    final ReaderContext context = searcher.getTopReaderContext();
+    final int weightDocFreq;
+    final PerReaderTermState termState;
+    if (perReaderTermState == null || perReaderTermState.topReaderContext != context) {
+      // make TermQuery single-pass if we don't have a PRTS or if the context differs!
+      termState = PerReaderTermState.build(context, term, true); // cache term lookups!
+      // we must not ignore the given docFreq - if set use the given value
+      weightDocFreq = docFreq == -1 ? termState.docFreq() : docFreq;
+    } else {
+     // PRTS was pre-build for this IS
+     termState = this.perReaderTermState;
+     weightDocFreq = docFreq;
+    }
+    
+    return new TermWeight(searcher, termState, weightDocFreq);
   }
 
   @Override

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TermScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TermScorer.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TermScorer.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TermScorer.java Wed Feb  9 09:35:27 2011
@@ -38,7 +38,8 @@ final class TermScorer extends Scorer {
   private int[] docs;
   private int[] freqs;
   private final DocsEnum.BulkReadResult bulkResult;
-
+  private final Similarity similarity;
+  
   /**
    * Construct a <code>TermScorer</code>.
    * 
@@ -53,15 +54,15 @@ final class TermScorer extends Scorer {
    *          The field norms of the document fields for the <code>Term</code>.
    */
   TermScorer(Weight weight, DocsEnum td, Similarity similarity, byte[] norms) {
-    super(similarity, weight);
-    
+    super(weight);
+    this.similarity = similarity;
     this.docsEnum = td;
     this.norms = norms;
     this.weightValue = weight.getValue();
     bulkResult = td.getBulkResult();
 
     for (int i = 0; i < SCORE_CACHE_SIZE; i++)
-      scoreCache[i] = getSimilarity().tf(i) * weightValue;
+      scoreCache[i] = similarity.tf(i) * weightValue;
   }
 
   @Override
@@ -77,7 +78,7 @@ final class TermScorer extends Scorer {
 
   // firstDocID is ignored since nextDoc() sets 'doc'
   @Override
-  protected boolean score(Collector c, int end, int firstDocID) throws IOException {
+  public boolean score(Collector c, int end, int firstDocID) throws IOException {
     c.setScorer(this);
     while (doc < end) {                           // for docs in window
       c.collect(doc);                      // collect score
@@ -136,9 +137,9 @@ final class TermScorer extends Scorer {
     float raw =                                   // compute tf(f)*weight
       freq < SCORE_CACHE_SIZE                        // check cache
       ? scoreCache[freq]                             // cache hit
-      : getSimilarity().tf(freq)*weightValue;        // cache miss
+      : similarity.tf(freq)*weightValue;        // cache miss
 
-    return norms == null ? raw : raw * getSimilarity().decodeNormValue(norms[doc]); // normalize for field
+    return norms == null ? raw : raw * similarity.decodeNormValue(norms[doc]); // normalize for field
   }
 
   /**

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TimeLimitingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TimeLimitingCollector.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TimeLimitingCollector.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TimeLimitingCollector.java Wed Feb  9 09:35:27 2011
@@ -19,7 +19,7 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.util.ThreadInterruptedException;
 
 /**
@@ -213,9 +213,9 @@ public class TimeLimitingCollector exten
   }
   
   @Override
-  public void setNextReader(IndexReader reader, int base) throws IOException {
-    collector.setNextReader(reader, base);
-    this.docBase = base;
+  public void setNextReader(AtomicReaderContext context) throws IOException {
+    collector.setNextReader(context);
+    this.docBase = context.docBase;
   }
   
   @Override

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopDocs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopDocs.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopDocs.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopDocs.java Wed Feb  9 09:35:27 2011
@@ -18,8 +18,8 @@ package org.apache.lucene.search;
  */
 
 /** Represents hits returned by {@link
- * Searcher#search(Query,Filter,int)} and {@link
- * Searcher#search(Query,int)}. */
+ * IndexSearcher#search(Query,Filter,int)} and {@link
+ * IndexSearcher#search(Query,int)}. */
 public class TopDocs implements java.io.Serializable {
   /** The total number of hits for the query.
   */

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopFieldCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopFieldCollector.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopFieldCollector.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopFieldCollector.java Wed Feb  9 09:35:27 2011
@@ -19,7 +19,7 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.search.FieldValueHitQueue.Entry;
 import org.apache.lucene.util.PriorityQueue;
 
@@ -92,9 +92,9 @@ public abstract class TopFieldCollector 
     }
     
     @Override
-    public void setNextReader(IndexReader reader, int docBase) throws IOException {
-      this.docBase = docBase;
-      queue.setComparator(0, comparator.setNextReader(reader, docBase));
+    public void setNextReader(AtomicReaderContext context) throws IOException {
+      this.docBase = context.docBase;
+      queue.setComparator(0, comparator.setNextReader(context));
       comparator = queue.firstComparator;
     }
     
@@ -447,10 +447,10 @@ public abstract class TopFieldCollector 
     }
 
     @Override
-    public void setNextReader(IndexReader reader, int docBase) throws IOException {
-      this.docBase = docBase;
+    public void setNextReader(AtomicReaderContext context) throws IOException {
+      docBase = context.docBase;
       for (int i = 0; i < comparators.length; i++) {
-        queue.setComparator(i, comparators[i].setNextReader(reader, docBase));
+        queue.setComparator(i, comparators[i].setNextReader(context));
       }
     }
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopFieldDocs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopFieldDocs.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopFieldDocs.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopFieldDocs.java Wed Feb  9 09:35:27 2011
@@ -19,7 +19,7 @@ package org.apache.lucene.search;
 
 
 /** Represents hits returned by {@link
- * Searcher#search(Query,Filter,int,Sort)}.
+ * IndexSearcher#search(Query,Filter,int,Sort)}.
  */
 public class TopFieldDocs
 extends TopDocs {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopScoreDocCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopScoreDocCollector.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopScoreDocCollector.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopScoreDocCollector.java Wed Feb  9 09:35:27 2011
@@ -19,7 +19,7 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 
 /**
  * A {@link Collector} implementation that collects the top-scoring hits,
@@ -155,8 +155,8 @@ public abstract class TopScoreDocCollect
   }
   
   @Override
-  public void setNextReader(IndexReader reader, int base) {
-    docBase = base;
+  public void setNextReader(AtomicReaderContext context) {
+    docBase = context.docBase;
   }
   
   @Override

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopTermsRewrite.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopTermsRewrite.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopTermsRewrite.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopTermsRewrite.java Wed Feb  9 09:35:27 2011
@@ -25,9 +25,11 @@ import java.util.Comparator;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.PerReaderTermState;
 
 /**
  * Base rewrite method for collecting only the top terms
@@ -78,14 +80,14 @@ public abstract class TopTermsRewrite<Q 
         this.termComp = termsEnum.getComparator();
         // lazy init the initial ScoreTerm because comparator is not known on ctor:
         if (st == null)
-          st = new ScoreTerm(this.termComp);
+          st = new ScoreTerm(this.termComp, new PerReaderTermState(topReaderContext));
         boostAtt = termsEnum.attributes().addAttribute(BoostAttribute.class);
       }
     
       @Override
-      public boolean collect(BytesRef bytes) {
+      public boolean collect(BytesRef bytes) throws IOException {
         final float boost = boostAtt.getBoost();
-        // ignore uncompetetive hits
+        // ignore uncompetitive hits
         if (stQueue.size() == maxSize) {
           final ScoreTerm t = stQueue.peek();
           if (boost < t.boost)
@@ -94,23 +96,27 @@ public abstract class TopTermsRewrite<Q 
             return true;
         }
         ScoreTerm t = visitedTerms.get(bytes);
+        final TermState state = termsEnum.termState();
+        assert state != null;
         if (t != null) {
           // if the term is already in the PQ, only update docFreq of term in PQ
-          t.docFreq += termsEnum.docFreq();
           assert t.boost == boost : "boost should be equal in all segment TermsEnums";
+          t.termState.register(state, readerContext.ord, termsEnum.docFreq());
         } else {
           // add new entry in PQ, we must clone the term, else it may get overwritten!
           st.bytes.copy(bytes);
           st.boost = boost;
-          st.docFreq = termsEnum.docFreq();
           visitedTerms.put(st.bytes, st);
+          assert st.termState.docFreq() == 0;
+          st.termState.register(state, readerContext.ord, termsEnum.docFreq());
           stQueue.offer(st);
           // possibly drop entries from queue
           if (stQueue.size() > maxSize) {
             st = stQueue.poll();
             visitedTerms.remove(st.bytes);
+            st.termState.clear(); // reset the termstate! 
           } else {
-            st = new ScoreTerm(termComp);
+            st = new ScoreTerm(termComp, new PerReaderTermState(topReaderContext));
           }
           assert stQueue.size() <= maxSize : "the PQ size must be limited to maxSize";
           // set maxBoostAtt with values to help FuzzyTermsEnum to optimize
@@ -120,6 +126,7 @@ public abstract class TopTermsRewrite<Q 
             maxBoostAtt.setCompetitiveTerm(t.bytes);
           }
         }
+       
         return true;
       }
     });
@@ -130,8 +137,8 @@ public abstract class TopTermsRewrite<Q 
     ArrayUtil.quickSort(scoreTerms, scoreTermSortByTermComp);
     for (final ScoreTerm st : scoreTerms) {
       final Term term = placeholderTerm.createTerm(st.bytes);
-      assert reader.docFreq(term) == st.docFreq;
-      addClause(q, term, st.docFreq, query.getBoost() * st.boost); // add to query
+      assert reader.docFreq(term) == st.termState.docFreq() : "reader DF is " + reader.docFreq(term) + " vs " + st.termState.docFreq();
+      addClause(q, term, st.termState.docFreq(), query.getBoost() * st.boost, st.termState); // add to query
     }
     query.incTotalNumberOfTerms(scoreTerms.length);
     return q;
@@ -147,7 +154,7 @@ public abstract class TopTermsRewrite<Q 
     if (this == obj) return true;
     if (obj == null) return false;
     if (getClass() != obj.getClass()) return false;
-    final TopTermsRewrite other = (TopTermsRewrite) obj;
+    final TopTermsRewrite<?> other = (TopTermsRewrite<?>) obj;
     if (size != other.size) return false;
     return true;
   }
@@ -163,13 +170,12 @@ public abstract class TopTermsRewrite<Q 
 
   static final class ScoreTerm implements Comparable<ScoreTerm> {
     public final Comparator<BytesRef> termComp;
-
     public final BytesRef bytes = new BytesRef();
     public float boost;
-    public int docFreq;
-    
-    public ScoreTerm(Comparator<BytesRef> termComp) {
+    public final PerReaderTermState termState;
+    public ScoreTerm(Comparator<BytesRef> termComp, PerReaderTermState termState) {
       this.termComp = termComp;
+      this.termState = termState;
     }
     
     public int compareTo(ScoreTerm other) {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TotalHitCountCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TotalHitCountCollector.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TotalHitCountCollector.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TotalHitCountCollector.java Wed Feb  9 09:35:27 2011
@@ -17,7 +17,7 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 
 /**
  * Just counts the total number of hits.
@@ -31,16 +31,20 @@ public class TotalHitCountCollector exte
     return totalHits;
   }
 
+  @Override
   public void setScorer(Scorer scorer) {
   }
 
+  @Override
   public void collect(int doc) {
     totalHits++;
   }
 
-  public void setNextReader(IndexReader reader, int docBase) {
+  @Override
+  public void setNextReader(AtomicReaderContext context) {
   }
 
+  @Override
   public boolean acceptsDocsOutOfOrder() {
     return true;
   }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Weight.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Weight.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Weight.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Weight.java Wed Feb  9 09:35:27 2011
@@ -21,27 +21,35 @@ import java.io.IOException;
 import java.io.Serializable;
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.IndexReader.ReaderContext;
 
 /**
  * Expert: Calculate query weights and build query scorers.
  * <p>
- * 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
+ * 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 IndexSearcher} dependent state of the query should reside in the
  * {@link Weight}. <br>
  * {@link IndexReader} dependent state should reside in the {@link Scorer}.
  * <p>
+ * Since {@link Weight} creates {@link Scorer} instances for a given
+ * {@link AtomicReaderContext} ({@link #scorer(IndexReader.AtomicReaderContext, ScorerContext)})
+ * callers must maintain the relationship between the searcher's top-level
+ * {@link ReaderContext} and the context used to create a {@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)}).
+ * <code>IndexSearcher</code> ({@link Query#createWeight(IndexSearcher)}).
  * <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
+ * {@link SimilarityProvider#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,boolean,boolean)}.
+ * <li>A <code>Scorer</code> is constructed by
+ * {@link #scorer(IndexReader.AtomicReaderContext, ScorerContext)}.
  * </ol>
  * 
  * @since 2.9
@@ -51,12 +59,12 @@ public abstract class Weight implements 
   /**
    * An explanation of the score computation for the named document.
    * 
-   * @param reader sub-reader containing the give doc
-   * @param doc
+   * @param context the readers context to create the {@link Explanation} for.
+   * @param doc the document's id relative to the given context's reader
    * @return an Explanation for the score
-   * @throws IOException
+   * @throws IOException if an {@link IOException} occurs
    */
-  public abstract Explanation explain(IndexReader reader, int doc) throws IOException;
+  public abstract Explanation explain(AtomicReaderContext context, int doc) throws IOException;
 
   /** The query that this concerns. */
   public abstract Query getQuery();
@@ -79,25 +87,13 @@ public abstract class Weight implements 
    * <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.
+   * @param context
+   *          the {@link AtomicReaderContext} for which to return the {@link Scorer}.
+   * @param scorerContext the {@link ScorerContext} holding the scores context variables
    * @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;
+  public abstract Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException;
   
   /** The sum of squared weights of contained query clauses. */
   public abstract float sumOfSquaredWeights() throws IOException;
@@ -106,7 +102,7 @@ public abstract class Weight implements 
    * 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
+   * {@link #scorer(IndexReader.AtomicReaderContext, ScorerContext)} to
    * create a matching {@link Scorer} instance for a given {@link Collector}, or
    * vice versa.
    * <p>
@@ -115,4 +111,82 @@ public abstract class Weight implements 
    */
   public boolean scoresDocsOutOfOrder() { return false; }
 
+  /**
+   * A struct like class encapsulating a scorer's context variables.
+   * ScorerContex is a strictly immutable struct that follows a
+   * <tt>create on modification</tt> pattern. If a context variable changes
+   * through one of the modifiers like {@link #topScorer(boolean)} a new
+   * {@link ScorerContext} instance is creates. If the modifier call doesn't
+   * change the instance the method call has no effect and the same instance is
+   * returned from the modifier.
+   * 
+   * @lucene.experimental
+   */
+  public static final class ScorerContext {
+   
+    /**
+     * 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.
+     */
+    public final boolean scoreDocsInOrder;
+    
+    /**
+     * if <code>true</code>, {@link Scorer#score(Collector)} will be called; if
+     * false, {@link Scorer#nextDoc()} and/or {@link Scorer#advance(int)} will
+     * be called instead.
+     */
+    public final boolean topScorer;
+    
+    
+    private static final ScorerContext DEFAULT_CONTEXT = new ScorerContext(true, false);
+
+    /**
+     * Returns a default {@link ScorerContext} template initialized with:
+     * <ul>
+     * <li>{@link #scoreDocsInOrder} = <code>true</code></li>
+     * <li>{@link #topScorer} = <code>false</code></li>
+     * </ul>
+     */
+    public static ScorerContext def() {
+      return DEFAULT_CONTEXT;
+    }
+    
+    private ScorerContext(boolean scoreDocsInOrder, boolean topScorer) {
+      this.scoreDocsInOrder = scoreDocsInOrder;
+      this.topScorer = topScorer;
+    }
+
+    /**
+     * Creates and returns a copy of this context with the given value for
+     * {@link #scoreDocsInOrder} and returns a new instance of
+     * {@link ScorerContext} iff the given value differs from the
+     * {@link #scoreDocsInOrder}. Otherwise, this method has no effect and
+     * returns this instance.
+     */
+    public ScorerContext scoreDocsInOrder(boolean scoreDocsInOrder) {
+      if (this.scoreDocsInOrder == scoreDocsInOrder) {
+        return this;
+      }
+      return new ScorerContext(scoreDocsInOrder, topScorer);
+    }
+    
+    /**
+     * Creates and returns a copy of this context with the given value for
+     * {@link #topScorer} and returns a new instance of
+     * {@link ScorerContext} iff the given value differs from the
+     * {@link #topScorer}. Otherwise, this method has no effect and
+     * returns this instance.
+     */
+    public ScorerContext topScorer(boolean topScorer) {
+      if (this.topScorer == topScorer) {
+        return this;
+      }
+      return new ScorerContext(scoreDocsInOrder, topScorer);
+    }
+  }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/cache/DocTermsIndexCreator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/cache/DocTermsIndexCreator.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/cache/DocTermsIndexCreator.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/cache/DocTermsIndexCreator.java Wed Feb  9 09:35:27 2011
@@ -24,6 +24,8 @@ import org.apache.lucene.index.DocsAndPo
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.MultiFields;
+import org.apache.lucene.index.OrdTermState;
+import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.DocIdSetIterator;
@@ -304,11 +306,6 @@ public class DocTermsIndexCreator extend
       }
 
       @Override
-      public void cacheCurrentTerm() throws IOException {
-        throw new UnsupportedOperationException();
-      }
-
-      @Override
       public BytesRef term() throws IOException {
         return term;
       }
@@ -324,6 +321,11 @@ public class DocTermsIndexCreator extend
       }
 
       @Override
+      public long totalTermFreq() {
+        return -1;
+      }
+
+      @Override
       public DocsEnum docs(Bits skipDocs, DocsEnum reuse) throws IOException {
         throw new UnsupportedOperationException();
       }
@@ -337,6 +339,19 @@ public class DocTermsIndexCreator extend
       public Comparator<BytesRef> getComparator() throws IOException {
         return BytesRef.getUTF8SortedAsUnicodeComparator();
       }
+
+      @Override
+      public void seek(BytesRef term, TermState state) throws IOException {
+        assert state != null && state instanceof OrdTermState;
+        this.seek(((OrdTermState)state).ord);
+      }
+
+      @Override
+      public TermState termState() throws IOException {
+        OrdTermState state = new OrdTermState();
+        state.ord = currentOrd;
+        return state;
+      }
     }
   }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/cache/EntryCreator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/cache/EntryCreator.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/cache/EntryCreator.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/cache/EntryCreator.java Wed Feb  9 09:35:27 2011
@@ -58,6 +58,7 @@ public abstract class EntryCreator<T> im
   // This can be removed
   //------------------------------------------------------------------------
 
+  @Override
   public boolean equals(Object obj) {
     if( obj instanceof EntryCreator ) {
       return getCacheKey().equals( ((EntryCreator)obj).getCacheKey() );

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/CustomScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/CustomScoreQuery.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/CustomScoreQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/CustomScoreQuery.java Wed Feb  9 09:35:27 2011
@@ -22,6 +22,7 @@ import java.util.Set;
 import java.util.Arrays;
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.ComplexExplanation;
 import org.apache.lucene.search.Explanation;
@@ -29,7 +30,6 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Weight;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Similarity;
 import org.apache.lucene.util.ToStringUtils;
 
 /**
@@ -52,7 +52,7 @@ public class CustomScoreQuery extends Qu
   
   /**
    * Create a CustomScoreQuery over input subQuery.
-   * @param subQuery the sub query whose scored is being customed. Must not be null. 
+   * @param subQuery the sub query whose scored is being customized. Must not be null. 
    */
   public CustomScoreQuery(Query subQuery) {
     this(subQuery, new ValueSourceQuery[0]);
@@ -182,13 +182,11 @@ public class CustomScoreQuery extends Qu
   //=========================== W E I G H T ============================
   
   private class CustomWeight extends Weight {
-    Similarity similarity;
     Weight subQueryWeight;
     Weight[] valSrcWeights;
     boolean qStrict;
 
     public CustomWeight(IndexSearcher searcher) throws IOException {
-      this.similarity = getSimilarity(searcher);
       this.subQueryWeight = subQuery.weight(searcher);
       this.valSrcWeights = new Weight[valSrcQueries.length];
       for(int i = 0; i < valSrcQueries.length; i++) {
@@ -239,40 +237,40 @@ public class CustomScoreQuery extends Qu
     }
 
     @Override
-    public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
+    public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
       // Pass true for "scoresDocsInOrder", because we
       // require in-order scoring, even if caller does not,
       // since we call advance on the valSrcScorers.  Pass
       // false for "topScorer" because we will not invoke
       // score(Collector) on these scorers:
-      Scorer subQueryScorer = subQueryWeight.scorer(reader, true, false);
+      Scorer subQueryScorer = subQueryWeight.scorer(context, ScorerContext.def());
       if (subQueryScorer == null) {
         return null;
       }
       Scorer[] valSrcScorers = new Scorer[valSrcWeights.length];
       for(int i = 0; i < valSrcScorers.length; i++) {
-         valSrcScorers[i] = valSrcWeights[i].scorer(reader, true, topScorer);
+         valSrcScorers[i] = valSrcWeights[i].scorer(context, scorerContext.scoreDocsInOrder(true));
       }
-      return new CustomScorer(similarity, reader, this, subQueryScorer, valSrcScorers);
+      return new CustomScorer(context.reader, this, subQueryScorer, valSrcScorers);
     }
 
     @Override
-    public Explanation explain(IndexReader reader, int doc) throws IOException {
-      Explanation explain = doExplain(reader, doc);
+    public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
+      Explanation explain = doExplain(context, doc);
       return explain == null ? new Explanation(0.0f, "no matching docs") : explain;
     }
     
-    private Explanation doExplain(IndexReader reader, int doc) throws IOException {
-      Explanation subQueryExpl = subQueryWeight.explain(reader, doc);
+    private Explanation doExplain(AtomicReaderContext info, int doc) throws IOException {
+      Explanation subQueryExpl = subQueryWeight.explain(info, doc);
       if (!subQueryExpl.isMatch()) {
         return subQueryExpl;
       }
       // match
       Explanation[] valSrcExpls = new Explanation[valSrcWeights.length];
       for(int i = 0; i < valSrcWeights.length; i++) {
-        valSrcExpls[i] = valSrcWeights[i].explain(reader, doc);
+        valSrcExpls[i] = valSrcWeights[i].explain(info, doc);
       }
-      Explanation customExp = CustomScoreQuery.this.getCustomScoreProvider(reader).customExplain(doc,subQueryExpl,valSrcExpls);
+      Explanation customExp = CustomScoreQuery.this.getCustomScoreProvider(info.reader).customExplain(doc,subQueryExpl,valSrcExpls);
       float sc = getValue() * customExp.getValue();
       Explanation res = new ComplexExplanation(
         true, sc, CustomScoreQuery.this.toString() + ", product of:");
@@ -302,9 +300,9 @@ public class CustomScoreQuery extends Qu
     private float vScores[]; // reused in score() to avoid allocating this array for each doc 
 
     // constructor
-    private CustomScorer(Similarity similarity, IndexReader reader, CustomWeight w,
+    private CustomScorer(IndexReader reader, CustomWeight w,
         Scorer subQueryScorer, Scorer[] valSrcScorers) throws IOException {
-      super(similarity,w);
+      super(w);
       this.qWeight = w.getValue();
       this.subQueryScorer = subQueryScorer;
       this.valSrcScorers = valSrcScorers;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/FieldCacheSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/FieldCacheSource.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/FieldCacheSource.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/FieldCacheSource.java Wed Feb  9 09:35:27 2011
@@ -20,6 +20,7 @@ package org.apache.lucene.search.functio
 import java.io.IOException;
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.search.FieldCache;
 
 /**
@@ -55,8 +56,8 @@ public abstract class FieldCacheSource e
 
   /* (non-Javadoc) @see org.apache.lucene.search.function.ValueSource#getValues(org.apache.lucene.index.IndexReader) */
   @Override
-  public final DocValues getValues(IndexReader reader) throws IOException {
-    return getCachedFieldValues(FieldCache.DEFAULT, field, reader);
+  public final DocValues getValues(AtomicReaderContext context) throws IOException {
+    return getCachedFieldValues(FieldCache.DEFAULT, field, context.reader);
   }
 
   /* (non-Javadoc) @see org.apache.lucene.search.function.ValueSource#description() */

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/MultiValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/MultiValueSource.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/MultiValueSource.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/MultiValueSource.java Wed Feb  9 09:35:27 2011
@@ -20,6 +20,9 @@ package org.apache.lucene.search.functio
 import java.io.IOException;
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.IndexReader.ReaderContext;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.util.ReaderUtil;
 
@@ -44,16 +47,17 @@ public final class MultiValueSource exte
   }
 
   @Override
-  public DocValues getValues(IndexReader reader) throws IOException {
-
-    IndexReader[] subReaders = reader.getSequentialSubReaders();
-    if (subReaders != null) {
-      // This is a composite reader
-      return new MultiDocValues(subReaders);
-    } else {
+  public DocValues getValues(AtomicReaderContext context) throws IOException {
       // Already an atomic reader -- just delegate
-      return other.getValues(reader);
+      return other.getValues(context);
+  }
+  
+  @Override
+  public DocValues getValues(ReaderContext context) throws IOException {
+    if (context.isAtomic) {
+      return getValues((AtomicReaderContext) context);
     }
+    return new MultiDocValues(ReaderUtil.leaves(context));
   }
 
   @Override
@@ -78,59 +82,56 @@ public final class MultiValueSource exte
   private final class MultiDocValues extends DocValues {
 
     final DocValues[] docValues;
-    final int[] docStarts;
+    final AtomicReaderContext[] leaves;
 
-    MultiDocValues(IndexReader[] subReaders) throws IOException {
-      docValues = new DocValues[subReaders.length];
-      docStarts = new int[subReaders.length];
-      int base = 0;
-      for(int i=0;i<subReaders.length;i++) {
-        docValues[i] = other.getValues(subReaders[i]);
-        docStarts[i] = base;
-        base += subReaders[i].maxDoc();
+    MultiDocValues(AtomicReaderContext[] leaves) throws IOException {
+      this.leaves = leaves;
+      docValues = new DocValues[leaves.length];
+      for(int i=0;i<leaves.length;i++) {
+        docValues[i] = other.getValues(leaves[i]);
       }
     }
     
     @Override
     public float floatVal(int doc) {
-      final int n = ReaderUtil.subIndex(doc, docStarts);
-      return docValues[n].floatVal(doc-docStarts[n]);
+      final int n = ReaderUtil.subIndex(doc, leaves);
+      return docValues[n].floatVal(doc-leaves[n].docBase);
     }
 
     @Override
     public int intVal(int doc) {
-      final int n = ReaderUtil.subIndex(doc, docStarts);
-      return docValues[n].intVal(doc-docStarts[n]);
+      final int n = ReaderUtil.subIndex(doc, leaves);
+      return docValues[n].intVal(doc-leaves[n].docBase);
     }
 
     @Override
     public long longVal(int doc) {
-      final int n = ReaderUtil.subIndex(doc, docStarts);
-      return docValues[n].longVal(doc-docStarts[n]);
+      final int n = ReaderUtil.subIndex(doc, leaves);
+      return docValues[n].longVal(doc-leaves[n].docBase);
     }
 
     @Override
     public double doubleVal(int doc) {
-      final int n = ReaderUtil.subIndex(doc, docStarts);
-      return docValues[n].doubleVal(doc-docStarts[n]);
+      final int n = ReaderUtil.subIndex(doc, leaves);
+      return docValues[n].doubleVal(doc-leaves[n].docBase);
     }
 
     @Override
     public String strVal(int doc) {
-      final int n = ReaderUtil.subIndex(doc, docStarts);
-      return docValues[n].strVal(doc-docStarts[n]);
+      final int n = ReaderUtil.subIndex(doc, leaves);
+      return docValues[n].strVal(doc-leaves[n].docBase);
     }
 
     @Override
     public String toString(int doc) {
-      final int n = ReaderUtil.subIndex(doc, docStarts);
-      return docValues[n].toString(doc-docStarts[n]);
+      final int n = ReaderUtil.subIndex(doc, leaves);
+      return docValues[n].toString(doc-leaves[n].docBase);
     }
 
     @Override
     public Explanation explain(int doc) {
-      final int n = ReaderUtil.subIndex(doc, docStarts);
-      return docValues[n].explain(doc-docStarts[n]);
+      final int n = ReaderUtil.subIndex(doc, leaves);
+      return docValues[n].explain(doc-leaves[n].docBase);
     }
   }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/OrdFieldSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/OrdFieldSource.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/OrdFieldSource.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/OrdFieldSource.java Wed Feb  9 09:35:27 2011
@@ -17,7 +17,7 @@
 
 package org.apache.lucene.search.function;
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.search.FieldCache;
 import org.apache.lucene.search.FieldCache.DocTermsIndex;
 
@@ -69,8 +69,8 @@ public class OrdFieldSource extends Valu
 
   /*(non-Javadoc) @see org.apache.lucene.search.function.ValueSource#getValues(org.apache.lucene.index.IndexReader) */
   @Override
-  public DocValues getValues(IndexReader reader) throws IOException {
-    final DocTermsIndex termsIndex = FieldCache.DEFAULT.getTermsIndex(reader, field);
+  public DocValues getValues(AtomicReaderContext context) throws IOException {
+    final DocTermsIndex termsIndex = FieldCache.DEFAULT.getTermsIndex(context.reader, field);
     return new DocValues() {
       /*(non-Javadoc) @see org.apache.lucene.search.function.DocValues#floatVal(int) */
       @Override
@@ -99,7 +99,9 @@ public class OrdFieldSource extends Valu
   /*(non-Javadoc) @see java.lang.Object#equals(java.lang.Object) */
   @Override
   public boolean equals(Object o) {
-    if (o.getClass() !=  OrdFieldSource.class) return false;
+    if (o == this) return true;
+    if (o == null) return false;
+    if (o.getClass() != OrdFieldSource.class) return false;
     OrdFieldSource other = (OrdFieldSource)o;
     return this.field.equals(other.field);
   }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/ReverseOrdFieldSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/ReverseOrdFieldSource.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/ReverseOrdFieldSource.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/ReverseOrdFieldSource.java Wed Feb  9 09:35:27 2011
@@ -17,7 +17,7 @@
 
 package org.apache.lucene.search.function;
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.search.FieldCache;
 
 import java.io.IOException;
@@ -69,8 +69,8 @@ public class ReverseOrdFieldSource exten
 
   /*(non-Javadoc) @see org.apache.lucene.search.function.ValueSource#getValues(org.apache.lucene.index.IndexReader) */
   @Override
-  public DocValues getValues(IndexReader reader) throws IOException {
-    final FieldCache.DocTermsIndex termsIndex = FieldCache.DEFAULT.getTermsIndex(reader, field);
+  public DocValues getValues(AtomicReaderContext context) throws IOException {
+    final FieldCache.DocTermsIndex termsIndex = FieldCache.DEFAULT.getTermsIndex(context.reader, field);
 
     final int end = termsIndex.numOrd();
 
@@ -107,7 +107,9 @@ public class ReverseOrdFieldSource exten
   /*(non-Javadoc) @see java.lang.Object#equals(java.lang.Object) */
   @Override
   public boolean equals(Object o) {
-    if (o.getClass() !=  ReverseOrdFieldSource.class) return false;
+    if (o == this) return true;
+    if (o == null) return false;
+    if (o.getClass() != ReverseOrdFieldSource.class) return false;
     ReverseOrdFieldSource other = (ReverseOrdFieldSource)o;
     return this.field.equals(other.field); 
   }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/ValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/ValueSource.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/ValueSource.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/ValueSource.java Wed Feb  9 09:35:27 2011
@@ -17,7 +17,9 @@ package org.apache.lucene.search.functio
  * limitations under the License.
  */
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.IndexReader.CompositeReaderContext;
+import org.apache.lucene.index.IndexReader.ReaderContext;
 import org.apache.lucene.search.function.DocValues;
 
 import java.io.IOException;
@@ -39,11 +41,23 @@ public abstract class ValueSource implem
 
   /**
    * Return the DocValues used by the function query.
-   * @param reader the IndexReader used to read these values.
+   * @param context the IndexReader used to read these values.
    * If any caching is involved, that caching would also be IndexReader based.  
    * @throws IOException for any error.
    */
-  public abstract DocValues getValues(IndexReader reader) throws IOException;
+  public abstract DocValues getValues(AtomicReaderContext context) throws IOException;
+  
+  /**
+   * Return the DocValues used by the function query.
+   * @deprecated (4.0) This method is temporary, to ease the migration to segment-based
+   * searching. Please change your code to not pass {@link CompositeReaderContext} to these
+   * APIs. Use {@link #getValues(IndexReader.AtomicReaderContext)} instead
+   */
+  @Deprecated
+  public DocValues getValues(ReaderContext context) throws IOException {
+    return getValues((AtomicReaderContext) context);
+  }
+
 
   /** 
    * description of field, used in explain() 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/ValueSourceQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/ValueSourceQuery.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/ValueSourceQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/ValueSourceQuery.java Wed Feb  9 09:35:27 2011
@@ -19,6 +19,7 @@ package org.apache.lucene.search.functio
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.search.*;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.util.ToStringUtils;
 import org.apache.lucene.util.Bits;
@@ -63,12 +64,10 @@ public class ValueSourceQuery extends Qu
   }
 
   class ValueSourceWeight extends Weight {
-    Similarity similarity;
     float queryNorm;
     float queryWeight;
 
     public ValueSourceWeight(IndexSearcher searcher) {
-      this.similarity = getSimilarity(searcher);
     }
 
     /*(non-Javadoc) @see org.apache.lucene.search.Weight#getQuery() */
@@ -98,14 +97,14 @@ public class ValueSourceQuery extends Qu
     }
 
     @Override
-    public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
-      return new ValueSourceScorer(similarity, reader, this);
+    public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
+      return new ValueSourceScorer(context, this);
     }
 
     /*(non-Javadoc) @see org.apache.lucene.search.Weight#explain(org.apache.lucene.index.IndexReader, int) */
     @Override
-    public Explanation explain(IndexReader reader, int doc) throws IOException {
-      DocValues vals = valSrc.getValues(reader);
+    public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
+      DocValues vals = valSrc.getValues(context);
       float sc = queryWeight * vals.floatVal(doc);
 
       Explanation result = new ComplexExplanation(
@@ -132,11 +131,12 @@ public class ValueSourceQuery extends Qu
     private int doc = -1;
 
     // constructor
-    private ValueSourceScorer(Similarity similarity, IndexReader reader, ValueSourceWeight w) throws IOException {
-      super(similarity,w);
+    private ValueSourceScorer(AtomicReaderContext context, ValueSourceWeight w) throws IOException {
+      super(w);
+      final IndexReader reader = context.reader;
       qWeight = w.getValue();
       // this is when/where the values are first created.
-      vals = valSrc.getValues(reader);
+      vals = valSrc.getValues(context);
       delDocs = reader.getDeletedDocs();
       maxDoc = reader.maxDoc();
     }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/package.html?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/package.html (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/package.html Wed Feb  9 09:35:27 2011
@@ -36,8 +36,8 @@ Code to search indices.
 Search over indices.
 
 Applications usually call {@link
-org.apache.lucene.search.Searcher#search(Query,int)} or {@link
-org.apache.lucene.search.Searcher#search(Query,Filter,int)}.
+org.apache.lucene.search.IndexSearcher#search(Query,int)} or {@link
+org.apache.lucene.search.IndexSearcher#search(Query,Filter,int)}.
 
     <!-- FILL IN MORE HERE -->   
 </p>

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java Wed Feb  9 09:35:27 2011
@@ -17,7 +17,7 @@ package org.apache.lucene.search.payload
  * limitations under the License.
  */
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.IndexSearcher;
@@ -143,10 +143,9 @@ public class PayloadNearQuery extends Sp
     }
 
     @Override
-    public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder,
-        boolean topScorer) throws IOException {
-      return new PayloadNearSpanScorer(query.getSpans(reader), this,
-          similarity, reader.norms(query.getField()));
+    public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
+      return new PayloadNearSpanScorer(query.getSpans(context), this,
+          similarity, context.reader.norms(query.getField()));
     }
   }
 
@@ -154,7 +153,6 @@ public class PayloadNearQuery extends Sp
     Spans spans;
     protected float payloadScore;
     private int payloadsSeen;
-    Similarity similarity = getSimilarity();
 
     protected PayloadNearSpanScorer(Spans spans, Weight weight,
         Similarity similarity, byte[] norms) throws IOException {
@@ -212,7 +210,7 @@ public class PayloadNearQuery extends Sp
           payloadsSeen = 0;
           do {
             int matchLength = spans.end() - spans.start();
-            freq += getSimilarity().sloppyFreq(matchLength);
+            freq += similarity.sloppyFreq(matchLength);
             Spans[] spansArr = new Spans[1];
             spansArr[0] = spans;
             getPayloads(spansArr);            
@@ -221,6 +219,7 @@ public class PayloadNearQuery extends Sp
           return true;    	
     }
 
+    @Override
     public float score() throws IOException {
 
       return super.score()

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/payloads/PayloadSpanUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/payloads/PayloadSpanUtil.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/payloads/PayloadSpanUtil.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/payloads/PayloadSpanUtil.java Wed Feb  9 09:35:27 2011
@@ -24,6 +24,8 @@ import java.util.Iterator;
 import java.util.List;
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.IndexReader.ReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
@@ -38,6 +40,7 @@ import org.apache.lucene.search.spans.Sp
 import org.apache.lucene.search.spans.SpanQuery;
 import org.apache.lucene.search.spans.SpanTermQuery;
 import org.apache.lucene.search.spans.Spans;
+import org.apache.lucene.util.ReaderUtil;
 
 /**
  * Experimental class to get set of payloads for most standard Lucene queries.
@@ -48,14 +51,16 @@ import org.apache.lucene.search.spans.Sp
  * 
  */
 public class PayloadSpanUtil {
-  private IndexReader reader;
+  private ReaderContext context;
 
   /**
-   * @param reader
+   * @param context
    *          that contains doc with payloads to extract
+   *          
+   * @see IndexReader#getTopReaderContext()
    */
-  public PayloadSpanUtil(IndexReader reader) {
-    this.reader = reader;
+  public PayloadSpanUtil(ReaderContext context) {
+    this.context = context;
   }
 
   /**
@@ -169,15 +174,16 @@ public class PayloadSpanUtil {
 
   private void getPayloads(Collection<byte []> payloads, SpanQuery query)
       throws IOException {
-    Spans spans = query.getSpans(reader);
-
-    while (spans.next() == true) {
-      if (spans.isPayloadAvailable()) {
-        Collection<byte[]> payload = spans.getPayload();
-        for (byte [] bytes : payload) {
-          payloads.add(bytes);
+    final AtomicReaderContext[] leaves = ReaderUtil.leaves(context);
+    for (AtomicReaderContext atomicReaderContext : leaves) {
+      final Spans spans = query.getSpans(atomicReaderContext);
+      while (spans.next() == true) {
+        if (spans.isPayloadAvailable()) {
+          Collection<byte[]> payload = spans.getPayload();
+          for (byte [] bytes : payload) {
+            payloads.add(bytes);
+          }
         }
-
       }
     }
   }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java Wed Feb  9 09:35:27 2011
@@ -17,9 +17,9 @@ package org.apache.lucene.search.payload
  * limitations under the License.
  */
 
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.DocsAndPositionsEnum;
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Weight;
@@ -74,10 +74,9 @@ public class PayloadTermQuery extends Sp
     }
 
     @Override
-    public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder,
-        boolean topScorer) throws IOException {
-      return new PayloadTermSpanScorer((TermSpans) query.getSpans(reader),
-          this, similarity, reader.norms(query.getField()));
+    public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
+      return new PayloadTermSpanScorer((TermSpans) query.getSpans(context),
+          this, similarity, context.reader.norms(query.getField()));
     }
 
     protected class PayloadTermSpanScorer extends SpanScorer {
@@ -101,12 +100,11 @@ public class PayloadTermQuery extends Sp
         freq = 0.0f;
         payloadScore = 0;
         payloadsSeen = 0;
-        Similarity similarity1 = getSimilarity();
         while (more && doc == spans.doc()) {
           int matchLength = spans.end() - spans.start();
 
-          freq += similarity1.sloppyFreq(matchLength);
-          processPayload(similarity1);
+          freq += similarity.sloppyFreq(matchLength);
+          processPayload(similarity);
 
           more = spans.next();// this moves positions to the next match in this
                               // document

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java Wed Feb  9 09:35:27 2011
@@ -21,11 +21,11 @@ import java.io.IOException;
 import java.util.Set;
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Weight;
 import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Similarity;
 import org.apache.lucene.util.ToStringUtils;
 
 /**
@@ -92,8 +92,8 @@ public class FieldMaskingSpanQuery exten
   // ...this is done to be more consistent with things like SpanFirstQuery
   
   @Override
-  public Spans getSpans(IndexReader reader) throws IOException {
-    return maskedQuery.getSpans(reader);
+  public Spans getSpans(AtomicReaderContext context) throws IOException {
+    return maskedQuery.getSpans(context);
   }
 
   @Override
@@ -107,11 +107,6 @@ public class FieldMaskingSpanQuery exten
   }
 
   @Override
-  public Similarity getSimilarity(IndexSearcher searcher) {
-    return maskedQuery.getSimilarity(searcher);
-  }
-
-  @Override
   public Query rewrite(IndexReader reader) throws IOException {
     FieldMaskingSpanQuery clone = null;
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java Wed Feb  9 09:35:27 2011
@@ -17,7 +17,7 @@ package org.apache.lucene.search.spans;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.util.ArrayUtil;
 
 import java.io.IOException;
@@ -77,11 +77,11 @@ public class NearSpansOrdered extends Sp
   private SpanNearQuery query;
   private boolean collectPayloads = true;
   
-  public NearSpansOrdered(SpanNearQuery spanNearQuery, IndexReader reader) throws IOException {
-    this(spanNearQuery, reader, true);
+  public NearSpansOrdered(SpanNearQuery spanNearQuery, AtomicReaderContext context) throws IOException {
+    this(spanNearQuery, context, true);
   }
 
-  public NearSpansOrdered(SpanNearQuery spanNearQuery, IndexReader reader, boolean collectPayloads)
+  public NearSpansOrdered(SpanNearQuery spanNearQuery, AtomicReaderContext context, boolean collectPayloads)
   throws IOException {
     if (spanNearQuery.getClauses().length < 2) {
       throw new IllegalArgumentException("Less than 2 clauses: "
@@ -94,7 +94,7 @@ public class NearSpansOrdered extends Sp
     matchPayload = new LinkedList<byte[]>();
     subSpansByDoc = new Spans[clauses.length];
     for (int i = 0; i < clauses.length; i++) {
-      subSpans[i] = clauses[i].getSpans(reader);
+      subSpans[i] = clauses[i].getSpans(context);
       subSpansByDoc[i] = subSpans[i]; // used in toSameDoc()
     }
     query = spanNearQuery; // kept for toString() only.

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java Wed Feb  9 09:35:27 2011
@@ -17,7 +17,7 @@ package org.apache.lucene.search.spans;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.util.PriorityQueue;
 
 import java.io.IOException;
@@ -131,7 +131,7 @@ public class NearSpansUnordered extends 
   }
 
 
-  public NearSpansUnordered(SpanNearQuery query, IndexReader reader)
+  public NearSpansUnordered(SpanNearQuery query, AtomicReaderContext context)
     throws IOException {
     this.query = query;
     this.slop = query.getSlop();
@@ -141,7 +141,7 @@ public class NearSpansUnordered extends 
     subSpans = new Spans[clauses.length];    
     for (int i = 0; i < clauses.length; i++) {
       SpansCell cell =
-        new SpansCell(clauses[i].getSpans(reader), i);
+        new SpansCell(clauses[i].getSpans(context), i);
       ordered.add(cell);
       subSpans[i] = cell.spans;
     }