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 [11/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/AutomatonTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/AutomatonTermsEnum.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/AutomatonTermsEnum.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/AutomatonTermsEnum.java Wed Feb  9 09:35:27 2011
@@ -27,6 +27,7 @@ import org.apache.lucene.util.automaton.
 import org.apache.lucene.util.automaton.ByteRunAutomaton;
 import org.apache.lucene.util.automaton.SpecialOperations;
 import org.apache.lucene.util.automaton.Transition;
+import org.apache.lucene.util.automaton.UTF32ToUTF8;
 
 /**
  * A FilteredTermsEnum that enumerates terms based upon what is accepted by a
@@ -46,8 +47,6 @@ import org.apache.lucene.util.automaton.
  * @lucene.experimental
  */
 public class AutomatonTermsEnum extends FilteredTermsEnum {
-  // the object-oriented form of the DFA
-  private final Automaton automaton;
   // a tableized array-based form of the DFA
   private final ByteRunAutomaton runAutomaton;
   // common suffix of the automaton
@@ -71,37 +70,20 @@ public class AutomatonTermsEnum extends 
   private final Comparator<BytesRef> termComp;
 
   /**
-   * Expert ctor:
    * Construct an enumerator based upon an automaton, enumerating the specified
    * field, working on a supplied TermsEnum
    * <p>
    * @lucene.experimental 
    * <p>
-   * @param runAutomaton pre-compiled ByteRunAutomaton
-   * @param finite true if the automaton accepts a finite language
+   * @param compiled CompiledAutomaton
    */
-  public AutomatonTermsEnum(ByteRunAutomaton runAutomaton,
-                     TermsEnum tenum,
-                     boolean finite, BytesRef commonSuffixRef)
-      throws IOException {
+  public AutomatonTermsEnum(TermsEnum tenum, CompiledAutomaton compiled) throws IOException {
     super(tenum);
-    this.automaton = runAutomaton.getAutomaton();
-    this.finite = finite;
+    this.finite = compiled.finite;
+    this.runAutomaton = compiled.runAutomaton;
+    this.commonSuffixRef = compiled.commonSuffixRef;
+    this.allTransitions = compiled.sortedTransitions;
 
-    this.runAutomaton = runAutomaton;
-    if (finite) {
-      // don't use suffix w/ finite DFAs
-      this.commonSuffixRef = null;
-    } else if (commonSuffixRef == null) {
-      // compute now
-      this.commonSuffixRef = SpecialOperations.getCommonSuffixBytesRef(automaton);
-    } else {
-      // precomputed
-      this.commonSuffixRef = commonSuffixRef;
-    }
-
-    // build a cache of sorted transitions for every state
-    allTransitions = this.automaton.getSortedTransitions();
     // used for path tracking, where each bit is a numbered state.
     visited = new long[runAutomaton.getSize()];
 
@@ -109,17 +91,6 @@ public class AutomatonTermsEnum extends 
   }
   
   /**
-   * Construct an enumerator based upon an automaton, enumerating the specified
-   * field, working on a supplied TermsEnum
-   * <p>
-   * It will automatically calculate whether or not the automaton is finite
-   */
-  public AutomatonTermsEnum(Automaton automaton, TermsEnum tenum)
-    throws IOException {
-    this(new ByteRunAutomaton(automaton), tenum, SpecialOperations.isFinite(automaton), null);
-  }
- 
-  /**
    * Returns true if the term matches the automaton. Also stashes away the term
    * to assist with smart enumeration.
    */
@@ -140,9 +111,9 @@ public class AutomatonTermsEnum extends 
   @Override
   protected BytesRef nextSeekTerm(final BytesRef term) throws IOException {
     if (term == null) {
-      seekBytesRef.copy("");
+      assert seekBytesRef.length == 0;
       // return the empty term, as its valid
-      if (runAutomaton.run(seekBytesRef.bytes, seekBytesRef.offset, seekBytesRef.length)) {   
+      if (runAutomaton.isAccept(runAutomaton.getInitialState())) {   
         return seekBytesRef;
       }
     } else {
@@ -151,25 +122,20 @@ public class AutomatonTermsEnum extends 
 
     // seek to the next possible string;
     if (nextString()) {
-      // reposition
-           
-      if (linear)
-        setLinear(infinitePosition);
-      return seekBytesRef;
+      return seekBytesRef;  // reposition
+    } else {
+      return null;          // no more possible strings can match
     }
-    // no more possible strings can match
-    return null;
   }
 
-  // this instance prevents unicode conversion during backtracking,
-  // we can just call setLinear once at the end.
-  int infinitePosition;
-
   /**
    * Sets the enum to operate in linear fashion, as we have found
-   * a looping transition at position
+   * a looping transition at position: we set an upper bound and 
+   * act like a TermRangeQuery for this portion of the term space.
    */
   private void setLinear(int position) {
+    assert linear == false;
+    
     int state = runAutomaton.getInitialState();
     int maxInterval = 0xff;
     for (int i = 0; i < position; i++) {
@@ -193,6 +159,8 @@ public class AutomatonTermsEnum extends 
     System.arraycopy(seekBytesRef.bytes, 0, linearUpperBound.bytes, 0, position);
     linearUpperBound.bytes[position] = (byte) maxInterval;
     linearUpperBound.length = length;
+    
+    linear = true;
   }
 
   private final IntsRef savedStates = new IntsRef(10);
@@ -226,8 +194,7 @@ public class AutomatonTermsEnum extends 
         states[pos+1] = nextState;
         // we found a loop, record it for faster enumeration
         if (!finite && !linear && visited[nextState] == curGen) {
-          linear = true;
-          infinitePosition = pos;
+          setLinear(pos);
         }
         state = nextState;
       }
@@ -313,15 +280,16 @@ public class AutomatonTermsEnum extends 
            */
           transition = allTransitions[state][0];
           state = transition.getDest().getNumber();
-          // we found a loop, record it for faster enumeration
-          if (!finite && !linear && visited[state] == curGen) {
-            linear = true;
-            infinitePosition = seekBytesRef.length;
-          }
+          
           // append the minimum transition
           seekBytesRef.grow(seekBytesRef.length + 1);
           seekBytesRef.length++;
           seekBytesRef.bytes[seekBytesRef.length - 1] = (byte) transition.getMin();
+          
+          // we found a loop, record it for faster enumeration
+          if (!finite && !linear && visited[state] == curGen) {
+            setLinear(seekBytesRef.length-1);
+          }
         }
         return true;
       }
@@ -350,4 +318,26 @@ public class AutomatonTermsEnum extends 
     }
     return -1; /* all solutions exhausted */
   }
+  
+  /**
+   * immutable class with everything this enum needs.
+   */
+  public static class CompiledAutomaton {
+    public final ByteRunAutomaton runAutomaton;
+    public final Transition[][] sortedTransitions;
+    public final BytesRef commonSuffixRef;
+    public final boolean finite;
+    
+    public CompiledAutomaton(Automaton automaton, boolean finite) {
+      Automaton utf8 = new UTF32ToUTF8().convert(automaton);
+      runAutomaton = new ByteRunAutomaton(utf8, true);
+      sortedTransitions = utf8.getSortedTransitions();
+      this.finite = finite;
+      if (finite) {
+        commonSuffixRef = null;
+      } else {
+        commonSuffixRef = SpecialOperations.getCommonSuffixBytesRef(utf8);
+      }
+    }
+  }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BooleanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BooleanQuery.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BooleanQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BooleanQuery.java Wed Feb  9 09:35:27 2011
@@ -18,6 +18,7 @@ package org.apache.lucene.search;
  */
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.util.ToStringUtils;
 import org.apache.lucene.search.BooleanClause.Occur;
@@ -62,46 +63,32 @@ public class BooleanQuery extends Query 
   }
 
   private ArrayList<BooleanClause> clauses = new ArrayList<BooleanClause>();
-  private boolean disableCoord;
+  private final boolean disableCoord;
 
   /** Constructs an empty boolean query. */
-  public BooleanQuery() {}
+  public BooleanQuery() {
+    disableCoord = false;
+  }
 
   /** Constructs an empty boolean query.
    *
-   * {@link Similarity#coord(int,int)} may be disabled in scoring, as
+   * {@link SimilarityProvider#coord(int,int)} may be disabled in scoring, as
    * appropriate. For example, this score factor does not make sense for most
    * automatically generated queries, like {@link WildcardQuery} and {@link
    * FuzzyQuery}.
    *
-   * @param disableCoord disables {@link Similarity#coord(int,int)} in scoring.
+   * @param disableCoord disables {@link SimilarityProvider#coord(int,int)} in scoring.
    */
   public BooleanQuery(boolean disableCoord) {
     this.disableCoord = disableCoord;
   }
 
-  /** Returns true iff {@link Similarity#coord(int,int)} is disabled in
+  /** Returns true iff {@link SimilarityProvider#coord(int,int)} is disabled in
    * scoring for this query instance.
    * @see #BooleanQuery(boolean)
    */
   public boolean isCoordDisabled() { return disableCoord; }
 
-  // Implement coord disabling.
-  // Inherit javadoc.
-  @Override
-  public Similarity getSimilarity(IndexSearcher searcher) {
-    Similarity result = super.getSimilarity(searcher);
-    if (disableCoord) {                           // disable coord as requested
-      result = new SimilarityDelegator(result) {
-          @Override
-          public float coord(int overlap, int maxOverlap) {
-            return 1.0f;
-          }
-        };
-    }
-    return result;
-  }
-
   /**
    * Specifies a minimum number of the optional BooleanClauses
    * which must be satisfied.
@@ -175,13 +162,15 @@ public class BooleanQuery extends Query 
    */
   protected class BooleanWeight extends Weight {
     /** The Similarity implementation. */
-    protected Similarity similarity;
+    protected SimilarityProvider similarityProvider;
     protected ArrayList<Weight> weights;
     protected int maxCoord;  // num optional + num required
+    private final boolean disableCoord;
 
-    public BooleanWeight(IndexSearcher searcher)
+    public BooleanWeight(IndexSearcher searcher, boolean disableCoord)
       throws IOException {
-      this.similarity = getSimilarity(searcher);
+      this.similarityProvider = searcher.getSimilarityProvider();
+      this.disableCoord = disableCoord;
       weights = new ArrayList<Weight>(clauses.size());
       for (int i = 0 ; i < clauses.size(); i++) {
         BooleanClause c = clauses.get(i);
@@ -212,6 +201,9 @@ public class BooleanQuery extends Query 
       return sum ;
     }
 
+    public float coord(int overlap, int maxOverlap) {
+      return similarityProvider.coord(overlap, maxOverlap);
+    }
 
     @Override
     public void normalize(float norm) {
@@ -223,7 +215,7 @@ public class BooleanQuery extends Query 
     }
 
     @Override
-    public Explanation explain(IndexReader reader, int doc)
+    public Explanation explain(AtomicReaderContext context, int doc)
       throws IOException {
       final int minShouldMatch =
         BooleanQuery.this.getMinimumNumberShouldMatch();
@@ -237,7 +229,7 @@ public class BooleanQuery extends Query 
       for (Iterator<Weight> wIter = weights.iterator(); wIter.hasNext();) {
         Weight w = wIter.next();
         BooleanClause c = cIter.next();
-        if (w.scorer(reader, true, true) == null) {
+        if (w.scorer(context, ScorerContext.def().scoreDocsInOrder(true).topScorer(true)) == null) {
           if (c.isRequired()) {
             fail = true;
             Explanation r = new Explanation(0.0f, "no match on required clause (" + c.getQuery().toString() + ")");
@@ -245,7 +237,7 @@ public class BooleanQuery extends Query 
           }
           continue;
         }
-        Explanation e = w.explain(reader, doc);
+        Explanation e = w.explain(context, doc);
         if (e.isMatch()) {
           if (!c.isProhibited()) {
             sumExpl.addDetail(e);
@@ -284,10 +276,10 @@ public class BooleanQuery extends Query 
       sumExpl.setMatch(0 < coord ? Boolean.TRUE : Boolean.FALSE);
       sumExpl.setValue(sum);
       
-      float coordFactor = similarity.coord(coord, maxCoord);
-      if (coordFactor == 1.0f)                      // coord is no-op
+      final float coordFactor = disableCoord ? 1.0f : coord(coord, maxCoord);
+      if (coordFactor == 1.0f) {
         return sumExpl;                             // eliminate wrapper
-      else {
+      } else {
         ComplexExplanation result = new ComplexExplanation(sumExpl.isMatch(),
                                                            sum*coordFactor,
                                                            "product of:");
@@ -299,7 +291,7 @@ public class BooleanQuery extends Query 
     }
 
     @Override
-    public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder, boolean topScorer)
+    public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext)
         throws IOException {
       List<Scorer> required = new ArrayList<Scorer>();
       List<Scorer> prohibited = new ArrayList<Scorer>();
@@ -307,7 +299,7 @@ public class BooleanQuery extends Query 
       Iterator<BooleanClause> cIter = clauses.iterator();
       for (Weight w  : weights) {
         BooleanClause c =  cIter.next();
-        Scorer subScorer = w.scorer(reader, true, false);
+        Scorer subScorer = w.scorer(context, ScorerContext.def());
         if (subScorer == null) {
           if (c.isRequired()) {
             return null;
@@ -322,8 +314,8 @@ public class BooleanQuery extends Query 
       }
       
       // Check if we can return a BooleanScorer
-      if (!scoreDocsInOrder && topScorer && required.size() == 0 && prohibited.size() < 32) {
-        return new BooleanScorer(this, similarity, minNrShouldMatch, optional, prohibited, maxCoord);
+      if (!scorerContext.scoreDocsInOrder && scorerContext.topScorer && required.size() == 0 && prohibited.size() < 32) {
+        return new BooleanScorer(this, disableCoord, minNrShouldMatch, optional, prohibited, maxCoord);
       }
       
       if (required.size() == 0 && optional.size() == 0) {
@@ -337,7 +329,7 @@ public class BooleanQuery extends Query 
       }
       
       // Return a BooleanScorer2
-      return new BooleanScorer2(this, similarity, minNrShouldMatch, required, prohibited, optional, maxCoord);
+      return new BooleanScorer2(this, disableCoord, minNrShouldMatch, required, prohibited, optional, maxCoord);
     }
     
     @Override
@@ -363,7 +355,7 @@ public class BooleanQuery extends Query 
 
   @Override
   public Weight createWeight(IndexSearcher searcher) throws IOException {
-    return new BooleanWeight(searcher);
+    return new BooleanWeight(searcher, disableCoord);
   }
 
   @Override

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BooleanScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BooleanScorer.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BooleanScorer.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BooleanScorer.java Wed Feb  9 09:35:27 2011
@@ -20,8 +20,9 @@ package org.apache.lucene.search;
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.search.BooleanQuery.BooleanWeight;
 
 /* Description from Doug Cutting (excerpted from
  * LUCENE-1483):
@@ -92,7 +93,7 @@ final class BooleanScorer extends Scorer
     }
     
     @Override
-    public void setNextReader(IndexReader reader, int docBase) {
+    public void setNextReader(AtomicReaderContext context) {
       // not needed by this implementation
     }
     
@@ -118,7 +119,7 @@ final class BooleanScorer extends Scorer
     int doc = NO_MORE_DOCS;
     int freq;
     
-    public BucketScorer() { super(null); }
+    public BucketScorer(Weight weight) { super(weight); }
     
     @Override
     public int advance(int target) throws IOException { return NO_MORE_DOCS; }
@@ -197,9 +198,9 @@ final class BooleanScorer extends Scorer
   private Bucket current;
   private int doc = -1;
   
-  BooleanScorer(Weight weight, Similarity similarity, int minNrShouldMatch,
+  BooleanScorer(BooleanWeight weight, boolean disableCoord, int minNrShouldMatch,
       List<Scorer> optionalScorers, List<Scorer> prohibitedScorers, int maxCoord) throws IOException {
-    super(similarity, weight);
+    super(weight);
     this.minNrShouldMatch = minNrShouldMatch;
 
     if (optionalScorers != null && optionalScorers.size() > 0) {
@@ -222,18 +223,17 @@ final class BooleanScorer extends Scorer
     }
 
     coordFactors = new float[optionalScorers.size() + 1];
-    Similarity sim = getSimilarity();
     for (int i = 0; i < coordFactors.length; i++) {
-      coordFactors[i] = sim.coord(i, maxCoord); 
+      coordFactors[i] = disableCoord ? 1.0f : weight.coord(i, maxCoord); 
     }
   }
 
   // firstDocID is ignored since nextDoc() initializes 'current'
   @Override
-  protected boolean score(Collector collector, int max, int firstDocID) throws IOException {
+  public boolean score(Collector collector, int max, int firstDocID) throws IOException {
     boolean more;
     Bucket tmp;
-    BucketScorer bs = new BucketScorer();
+    BucketScorer bs = new BucketScorer(weight);
     // The internal loop will set the score and doc before calling collect.
     collector.setScorer(bs);
     do {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BooleanScorer2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BooleanScorer2.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BooleanScorer2.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BooleanScorer2.java Wed Feb  9 09:35:27 2011
@@ -22,6 +22,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.search.BooleanQuery.BooleanWeight;
 
 /* See the description in BooleanScorer.java, comparing
  * BooleanScorer & BooleanScorer2 */
@@ -42,14 +43,12 @@ class BooleanScorer2 extends Scorer {
     int maxCoord = 0; // to be increased for each non prohibited scorer
     int nrMatchers; // to be increased by score() of match counting scorers.
     
-    void init() { // use after all scorers have been added.
+    void init(boolean disableCoord) { // use after all scorers have been added.
       coordFactors = new float[optionalScorers.size() + requiredScorers.size() + 1];
-      Similarity sim = getSimilarity();
       for (int i = 0; i < coordFactors.length; i++) {
-        coordFactors[i] = sim.coord(i, maxCoord);
+        coordFactors[i] = disableCoord ? 1.0f : ((BooleanWeight)weight).coord(i, maxCoord);
       }
     }
-    
   }
 
   private final Coordinator coordinator;
@@ -69,8 +68,11 @@ class BooleanScorer2 extends Scorer {
    * prohibited and optional scorers. In no required scorers are added, at least
    * one of the optional scorers will have to match during the search.
    * 
-   * @param similarity
-   *          The similarity to be used.
+   * @param weight
+   *          The BooleanWeight to be used.
+   * @param disableCoord
+   *          If this parameter is true, coordination level matching 
+   *          ({@link Similarity#coord(int, int)}) is not used.
    * @param minNrShouldMatch
    *          The minimum number of optional added scorers that should match
    *          during the search. In case no required scorers are added, at least
@@ -82,9 +84,9 @@ class BooleanScorer2 extends Scorer {
    * @param optional
    *          the list of optional scorers.
    */
-  public BooleanScorer2(Weight weight, Similarity similarity, int minNrShouldMatch,
+  public BooleanScorer2(BooleanWeight weight, boolean disableCoord, int minNrShouldMatch,
       List<Scorer> required, List<Scorer> prohibited, List<Scorer> optional, int maxCoord) throws IOException {
-    super(similarity, weight);
+    super(weight);
     if (minNrShouldMatch < 0) {
       throw new IllegalArgumentException("Minimum number of optional scorers should not be negative");
     }
@@ -96,8 +98,8 @@ class BooleanScorer2 extends Scorer {
     requiredScorers = required;    
     prohibitedScorers = prohibited;
     
-    coordinator.init();
-    countingSumScorer = makeCountingSumScorer();
+    coordinator.init(disableCoord);
+    countingSumScorer = makeCountingSumScorer(disableCoord);
   }
   
   /** Count a scorer as a single match. */
@@ -109,7 +111,7 @@ class BooleanScorer2 extends Scorer {
     private float lastDocScore = Float.NaN;
 
     SingleMatchScorer(Scorer scorer) {
-      super(scorer.getSimilarity());
+      super(scorer.weight);
       this.scorer = scorer;
     }
 
@@ -145,7 +147,7 @@ class BooleanScorer2 extends Scorer {
   private Scorer countingDisjunctionSumScorer(final List<Scorer> scorers,
       int minNrShouldMatch) throws IOException {
     // each scorer from the list counted as a single matcher
-    return new DisjunctionSumScorer(scorers, minNrShouldMatch) {
+    return new DisjunctionSumScorer(weight, scorers, minNrShouldMatch) {
       private int lastScoredDoc = -1;
       // Save the score of lastScoredDoc, so that we don't compute it more than
       // once in score().
@@ -164,12 +166,11 @@ class BooleanScorer2 extends Scorer {
     };
   }
 
-  private static final Similarity defaultSimilarity = Similarity.getDefault();
-
-  private Scorer countingConjunctionSumScorer(List<Scorer> requiredScorers) throws IOException {
+  private Scorer countingConjunctionSumScorer(boolean disableCoord,
+                                              List<Scorer> requiredScorers) throws IOException {
     // each scorer from the list counted as a single matcher
     final int requiredNrMatchers = requiredScorers.size();
-    return new ConjunctionScorer(defaultSimilarity, requiredScorers) {
+    return new ConjunctionScorer(weight, disableCoord ? 1.0f : ((BooleanWeight)weight).coord(requiredScorers.size(), requiredScorers.size()), requiredScorers) {
       private int lastScoredDoc = -1;
       // Save the score of lastScoredDoc, so that we don't compute it more than
       // once in score().
@@ -192,8 +193,9 @@ class BooleanScorer2 extends Scorer {
     };
   }
 
-  private Scorer dualConjunctionSumScorer(Scorer req1, Scorer req2) throws IOException { // non counting.
-    return new ConjunctionScorer(defaultSimilarity, req1, req2);
+  private Scorer dualConjunctionSumScorer(boolean disableCoord,
+                                          Scorer req1, Scorer req2) throws IOException { // non counting.
+    return new ConjunctionScorer(weight, disableCoord ? 1.0f : ((BooleanWeight)weight).coord(2, 2), req1, req2);
     // All scorers match, so defaultSimilarity always has 1 as
     // the coordination factor.
     // Therefore the sum of the scores of two scorers
@@ -203,13 +205,13 @@ class BooleanScorer2 extends Scorer {
   /** Returns the scorer to be used for match counting and score summing.
    * Uses requiredScorers, optionalScorers and prohibitedScorers.
    */
-  private Scorer makeCountingSumScorer() throws IOException { // each scorer counted as a single matcher
+  private Scorer makeCountingSumScorer(boolean disableCoord) throws IOException { // each scorer counted as a single matcher
     return (requiredScorers.size() == 0)
-          ? makeCountingSumScorerNoReq()
-          : makeCountingSumScorerSomeReq();
+      ? makeCountingSumScorerNoReq(disableCoord)
+      : makeCountingSumScorerSomeReq(disableCoord);
   }
 
-  private Scorer makeCountingSumScorerNoReq() throws IOException { // No required scorers
+  private Scorer makeCountingSumScorerNoReq(boolean disableCoord) throws IOException { // No required scorers
     // minNrShouldMatch optional scorers are required, but at least 1
     int nrOptRequired = (minNrShouldMatch < 1) ? 1 : minNrShouldMatch;
     Scorer requiredCountingSumScorer;
@@ -217,24 +219,26 @@ class BooleanScorer2 extends Scorer {
       requiredCountingSumScorer = countingDisjunctionSumScorer(optionalScorers, nrOptRequired);
     else if (optionalScorers.size() == 1)
       requiredCountingSumScorer = new SingleMatchScorer(optionalScorers.get(0));
-    else
-      requiredCountingSumScorer = countingConjunctionSumScorer(optionalScorers);
+    else {
+      requiredCountingSumScorer = countingConjunctionSumScorer(disableCoord, optionalScorers);
+    }
     return addProhibitedScorers(requiredCountingSumScorer);
   }
 
-  private Scorer makeCountingSumScorerSomeReq() throws IOException { // At least one required scorer.
+  private Scorer makeCountingSumScorerSomeReq(boolean disableCoord) throws IOException { // At least one required scorer.
     if (optionalScorers.size() == minNrShouldMatch) { // all optional scorers also required.
       ArrayList<Scorer> allReq = new ArrayList<Scorer>(requiredScorers);
       allReq.addAll(optionalScorers);
-      return addProhibitedScorers(countingConjunctionSumScorer(allReq));
+      return addProhibitedScorers(countingConjunctionSumScorer(disableCoord, allReq));
     } else { // optionalScorers.size() > minNrShouldMatch, and at least one required scorer
       Scorer requiredCountingSumScorer =
             requiredScorers.size() == 1
             ? new SingleMatchScorer(requiredScorers.get(0))
-            : countingConjunctionSumScorer(requiredScorers);
+            : countingConjunctionSumScorer(disableCoord, requiredScorers);
       if (minNrShouldMatch > 0) { // use a required disjunction scorer over the optional scorers
         return addProhibitedScorers( 
                       dualConjunctionSumScorer( // non counting
+                              disableCoord,
                               requiredCountingSumScorer,
                               countingDisjunctionSumScorer(
                                       optionalScorers,
@@ -261,7 +265,7 @@ class BooleanScorer2 extends Scorer {
           : new ReqExclScorer(requiredCountingSumScorer,
                               ((prohibitedScorers.size() == 1)
                                 ? prohibitedScorers.get(0)
-                                : new DisjunctionSumScorer(prohibitedScorers)));
+                                : new DisjunctionSumScorer(weight, prohibitedScorers)));
   }
 
   /** Scores and collects all matching documents.
@@ -276,7 +280,7 @@ class BooleanScorer2 extends Scorer {
   }
   
   @Override
-  protected boolean score(Collector collector, int max, int firstDocID) throws IOException {
+  public boolean score(Collector collector, int max, int firstDocID) throws IOException {
     doc = firstDocID;
     collector.setScorer(this);
     while (doc < max) {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BoostAttribute.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BoostAttribute.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BoostAttribute.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BoostAttribute.java Wed Feb  9 09:35:27 2011
@@ -21,13 +21,13 @@ import org.apache.lucene.util.Attribute;
 import org.apache.lucene.util.AttributeSource; // javadocs only
 import org.apache.lucene.index.TermsEnum; // javadocs only
 
-/** Add this {@link Attribute} to a {@link TermsEnum} returned by {@link MultiTermQuery#getTermsEnum(IndexReader,AttributeSource)}
+/** Add this {@link Attribute} to a {@link TermsEnum} returned by {@link MultiTermQuery#getTermsEnum(Terms,AttributeSource)}
  * and update the boost on each returned term. This enables to control the boost factor
  * for each matching term in {@link MultiTermQuery#SCORING_BOOLEAN_QUERY_REWRITE} or
  * {@link TopTermsRewrite} mode.
  * {@link FuzzyQuery} is using this to take the edit distance into account.
  * <p><b>Please note:</b> This attribute is intended to be added only by the TermsEnum
- * to itsself in its constructor and consumed by the {@link MultiTermQuery.RewriteMethod}.
+ * to itself in its constructor and consumed by the {@link MultiTermQuery.RewriteMethod}.
  * @lucene.internal
  */
 public interface BoostAttribute extends Attribute {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BoostAttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BoostAttributeImpl.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BoostAttributeImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BoostAttributeImpl.java Wed Feb  9 09:35:27 2011
@@ -37,20 +37,6 @@ public final class BoostAttributeImpl ex
   public void clear() {
     boost = 1.0f;
   }
-
-  @Override
-  public boolean equals(Object other) {
-    if (this == other)
-      return true;
-    if (other instanceof BoostAttributeImpl)
-      return ((BoostAttributeImpl) other).boost == boost;
-    return false;
-  }
-
-  @Override
-  public int hashCode() {
-    return Float.floatToIntBits(boost);
-  }
   
   @Override
   public void copyTo(AttributeImpl target) {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/CachingSpanFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/CachingSpanFilter.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/CachingSpanFilter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/CachingSpanFilter.java Wed Feb  9 09:35:27 2011
@@ -17,6 +17,7 @@ package org.apache.lucene.search;
 
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.util.Bits;
 
 import java.io.IOException;
@@ -60,15 +61,16 @@ public class CachingSpanFilter extends S
   }
 
   @Override
-  public DocIdSet getDocIdSet(IndexReader reader) throws IOException {
-    SpanFilterResult result = getCachedResult(reader);
+  public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
+    SpanFilterResult result = getCachedResult(context);
     return result != null ? result.getDocIdSet() : null;
   }
   
   // for testing
   int hitCount, missCount;
 
-  private SpanFilterResult getCachedResult(IndexReader reader) throws IOException {
+  private SpanFilterResult getCachedResult(AtomicReaderContext context) throws IOException {
+    final IndexReader reader = context.reader;
 
     final Object coreKey = reader.getCoreCacheKey();
     final Object delCoreKey = reader.hasDeletions() ? reader.getDeletedDocs() : coreKey;
@@ -80,7 +82,7 @@ public class CachingSpanFilter extends S
     }
 
     missCount++;
-    result = filter.bitSpans(reader);
+    result = filter.bitSpans(context);
 
     cache.put(coreKey, delCoreKey, result);
     return result;
@@ -88,8 +90,8 @@ public class CachingSpanFilter extends S
 
 
   @Override
-  public SpanFilterResult bitSpans(IndexReader reader) throws IOException {
-    return getCachedResult(reader);
+  public SpanFilterResult bitSpans(AtomicReaderContext context) throws IOException {
+    return getCachedResult(context);
   }
 
   @Override

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/CachingWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/CachingWrapperFilter.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/CachingWrapperFilter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/CachingWrapperFilter.java Wed Feb  9 09:35:27 2011
@@ -23,6 +23,7 @@ import java.util.Map;
 import java.util.WeakHashMap;
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.util.OpenBitSetDISI;
 import org.apache.lucene.util.Bits;
 
@@ -37,6 +38,9 @@ import org.apache.lucene.util.Bits;
  * {@link DeletesMode#DYNAMIC}).
  */
 public class CachingWrapperFilter extends Filter {
+  // TODO: make this filter aware of ReaderContext. a cached filter could 
+  // specify the actual readers key or something similar to indicate on which
+  // level of the readers hierarchy it should be cached.
   Filter filter;
 
   /**
@@ -191,8 +195,8 @@ public class CachingWrapperFilter extend
   int hitCount, missCount;
 
   @Override
-  public DocIdSet getDocIdSet(IndexReader reader) throws IOException {
-
+  public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
+    final IndexReader reader = context.reader;
     final Object coreKey = reader.getCoreCacheKey();
     final Object delCoreKey = reader.hasDeletions() ? reader.getDeletedDocs() : coreKey;
 
@@ -205,7 +209,7 @@ public class CachingWrapperFilter extend
     missCount++;
 
     // cache miss
-    docIdSet = docIdSetToCache(filter.getDocIdSet(reader), reader);
+    docIdSet = docIdSetToCache(filter.getDocIdSet(context), reader);
 
     if (docIdSet != null) {
       cache.put(coreKey, delCoreKey, docIdSet);

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Collector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Collector.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Collector.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Collector.java Wed Feb  9 09:35:27 2011
@@ -19,7 +19,8 @@ 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.index.IndexReader.ReaderContext;
 
 /**
  * <p>Expert: Collectors are primarily meant to be used to
@@ -98,8 +99,8 @@ import org.apache.lucene.index.IndexRead
  *     bits.set(doc + docBase);
  *   }
  * 
- *   public void setNextReader(IndexReader reader, int docBase) {
- *     this.docBase = docBase;
+ *   public void setNextReader(AtomicReaderContext context) {
+ *     this.docBase = context.docBase;
  *   }
  * });
  * </pre>
@@ -136,24 +137,23 @@ public abstract class Collector {
    * 
    * <p>
    * Note: This is called in an inner search loop. For good search performance,
-   * implementations of this method should not call {@link Searcher#doc(int)} or
+   * implementations of this method should not call {@link IndexSearcher#doc(int)} or
    * {@link org.apache.lucene.index.IndexReader#document(int)} on every hit.
    * Doing so can slow searches by an order of magnitude or more.
    */
   public abstract void collect(int doc) throws IOException;
 
   /**
-   * Called before collecting from each IndexReader. All doc ids in
-   * {@link #collect(int)} will correspond to reader.
+   * Called before collecting from each {@link AtomicReaderContext}. All doc ids in
+   * {@link #collect(int)} will correspond to {@link ReaderContext#reader}.
    * 
-   * Add docBase to the current IndexReaders internal document id to re-base ids
-   * in {@link #collect(int)}.
+   * Add {@link AtomicReaderContext#docBase} to the current  {@link ReaderContext#reader}'s
+   * internal document id to re-base ids in {@link #collect(int)}.
    * 
-   * @param reader
-   *          next IndexReader
-   * @param docBase
+   * @param context
+   *          next atomic reader context
    */
-  public abstract void setNextReader(IndexReader reader, int docBase) throws IOException;
+  public abstract void setNextReader(AtomicReaderContext context) throws IOException;
 
   /**
    * Return <code>true</code> if this collector does not

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ConjunctionScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ConjunctionScorer.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ConjunctionScorer.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ConjunctionScorer.java Wed Feb  9 09:35:27 2011
@@ -29,14 +29,14 @@ class ConjunctionScorer extends Scorer {
   private final float coord;
   private int lastDoc = -1;
 
-  public ConjunctionScorer(Similarity similarity, Collection<Scorer> scorers) throws IOException {
-    this(similarity, scorers.toArray(new Scorer[scorers.size()]));
+  public ConjunctionScorer(Weight weight, float coord, Collection<Scorer> scorers) throws IOException {
+    this(weight, coord, scorers.toArray(new Scorer[scorers.size()]));
   }
 
-  public ConjunctionScorer(Similarity similarity, Scorer... scorers) throws IOException {
-    super(similarity);
+  public ConjunctionScorer(Weight weight, float coord, Scorer... scorers) throws IOException {
+    super(weight);
     this.scorers = scorers;
-    coord = similarity.coord(scorers.length, scorers.length);
+    this.coord = coord;
     
     for (int i = 0; i < scorers.length; i++) {
       if (scorers[i].nextDoc() == NO_MORE_DOCS) {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ConstantScoreAutoRewrite.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ConstantScoreAutoRewrite.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ConstantScoreAutoRewrite.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ConstantScoreAutoRewrite.java Wed Feb  9 09:35:27 2011
@@ -21,9 +21,15 @@ import java.io.IOException;
 
 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.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash;
+import org.apache.lucene.util.PerReaderTermState;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.BytesRefHash.DirectBytesStartArray;
 
 class ConstantScoreAutoRewrite extends TermCollectingRewrite<BooleanQuery> {
 
@@ -71,8 +77,8 @@ class ConstantScoreAutoRewrite extends T
   }
   
   @Override
-  protected void addClause(BooleanQuery topLevel, Term term, int docFreq, float boost /*ignored*/) {
-    topLevel.add(new TermQuery(term, docFreq), BooleanClause.Occur.SHOULD);
+  protected void addClause(BooleanQuery topLevel, Term term, int docFreq, float boost /*ignored*/, PerReaderTermState states) {
+    topLevel.add(new TermQuery(term, states), BooleanClause.Occur.SHOULD);
   }
 
   @Override
@@ -98,9 +104,10 @@ class ConstantScoreAutoRewrite extends T
       final BytesRefHash pendingTerms = col.pendingTerms;
       final int sort[] = pendingTerms.sort(col.termsEnum.getComparator());
       for(int i = 0; i < size; i++) {
+        final int pos = sort[i];
         // docFreq is not used for constant score here, we pass 1
         // to explicitely set a fake value, so it's not calculated
-        addClause(bq, placeholderTerm.createTerm(pendingTerms.get(sort[i], new BytesRef())), 1, 1.0f);
+        addClause(bq, placeholderTerm.createTerm(pendingTerms.get(pos, new BytesRef())), 1, 1.0f, col.array.termState[pos]);
       }
       // Strip scores
       final Query result = new ConstantScoreQuery(bq);
@@ -123,12 +130,21 @@ class ConstantScoreAutoRewrite extends T
       
     @Override
     public boolean collect(BytesRef bytes) throws IOException {
-      pendingTerms.add(bytes);
+      int pos = pendingTerms.add(bytes);
       docVisitCount += termsEnum.docFreq();
       if (pendingTerms.size() >= termCountLimit || docVisitCount >= docCountCutoff) {
         hasCutOff = true;
         return false;
       }
+      
+      final TermState termState = termsEnum.termState();
+      assert termState != null;
+      if (pos < 0) {
+        pos = (-pos)-1;
+        array.termState[pos].register(termState, readerContext.ord, termsEnum.docFreq());
+      } else {
+        array.termState[pos] = new PerReaderTermState(topReaderContext, termState, readerContext.ord, termsEnum.docFreq());
+      }
       return true;
     }
     
@@ -137,7 +153,8 @@ class ConstantScoreAutoRewrite extends T
     TermsEnum termsEnum;
 
     final int docCountCutoff, termCountLimit;
-    final BytesRefHash pendingTerms = new BytesRefHash();
+    final TermStateByteStart array = new TermStateByteStart(16);
+    final BytesRefHash pendingTerms = new BytesRefHash(new ByteBlockPool(new ByteBlockPool.DirectAllocator()), 16, array);
   }
 
   @Override
@@ -166,4 +183,40 @@ class ConstantScoreAutoRewrite extends T
     
     return true;
   }
+  
+  /** Special implementation of BytesStartArray that keeps parallel arrays for {@link PerReaderTermState} */
+  static final class TermStateByteStart extends DirectBytesStartArray  {
+    PerReaderTermState[] termState;
+    
+    public TermStateByteStart(int initSize) {
+      super(initSize);
+    }
+
+    @Override
+    public int[] init() {
+      final int[] ord = super.init();
+      termState = new PerReaderTermState[ArrayUtil.oversize(ord.length, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+      assert termState.length >= ord.length;
+      return ord;
+    }
+
+    @Override
+    public int[] grow() {
+      final int[] ord = super.grow();
+      if (termState.length < ord.length) {
+        PerReaderTermState[] tmpTermState = new PerReaderTermState[ArrayUtil.oversize(ord.length, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+        System.arraycopy(termState, 0, tmpTermState, 0, termState.length);
+        termState = tmpTermState;
+      }      
+      assert termState.length >= ord.length;
+      return ord;
+    }
+
+    @Override
+    public int[] clear() {
+     termState = null;
+     return super.clear();
+    }
+    
+  }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java Wed Feb  9 09:35:27 2011
@@ -18,6 +18,7 @@ package org.apache.lucene.search;
  */
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.util.ToStringUtils;
 
@@ -96,12 +97,10 @@ public class ConstantScoreQuery extends 
 
   protected class ConstantWeight extends Weight {
     private final Weight innerWeight;
-    private final Similarity similarity;
     private float queryNorm;
     private float queryWeight;
     
     public ConstantWeight(IndexSearcher searcher) throws IOException {
-      this.similarity = getSimilarity(searcher);
       this.innerWeight = (query == null) ? null : query.createWeight(searcher);
     }
 
@@ -132,22 +131,22 @@ public class ConstantScoreQuery extends 
     }
 
     @Override
-    public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
+    public Scorer scorer(AtomicReaderContext context,  ScorerContext scorerContext) throws IOException {
       final DocIdSetIterator disi;
       if (filter != null) {
         assert query == null;
-        final DocIdSet dis = filter.getDocIdSet(reader);
+        final DocIdSet dis = filter.getDocIdSet(context);
         if (dis == null)
           return null;
         disi = dis.iterator();
       } else {
         assert query != null && innerWeight != null;
         disi =
-          innerWeight.scorer(reader, scoreDocsInOrder, topScorer);
+          innerWeight.scorer(context, scorerContext);
       }
       if (disi == null)
         return null;
-      return new ConstantScorer(similarity, disi, this);
+      return new ConstantScorer(disi, this);
     }
     
     @Override
@@ -156,8 +155,8 @@ public class ConstantScoreQuery extends 
     }
 
     @Override
-    public Explanation explain(IndexReader reader, int doc) throws IOException {
-      final Scorer cs = scorer(reader, true, false);
+    public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
+      final Scorer cs = scorer(context, ScorerContext.def());
       final boolean exists = (cs != null && cs.advance(doc) == doc);
 
       final ComplexExplanation result = new ComplexExplanation();
@@ -180,8 +179,8 @@ public class ConstantScoreQuery extends 
     final DocIdSetIterator docIdSetIterator;
     final float theScore;
 
-    public ConstantScorer(Similarity similarity, DocIdSetIterator docIdSetIterator, Weight w) throws IOException {
-      super(similarity,w);
+    public ConstantScorer(DocIdSetIterator docIdSetIterator, Weight w) throws IOException {
+      super(w);
       theScore = w.getValue();
       this.docIdSetIterator = docIdSetIterator;
     }
@@ -211,8 +210,7 @@ public class ConstantScoreQuery extends 
         @Override
         public void setScorer(Scorer scorer) throws IOException {
           // we must wrap again here, but using the scorer passed in as parameter:
-          collector.setScorer(new ConstantScorer(ConstantScorer.this.getSimilarity(),
-            scorer, ConstantScorer.this.weight));
+          collector.setScorer(new ConstantScorer(scorer, ConstantScorer.this.weight));
         }
         
         @Override
@@ -221,8 +219,8 @@ public class ConstantScoreQuery extends 
         }
         
         @Override
-        public void setNextReader(IndexReader reader, int docBase) throws IOException {
-          collector.setNextReader(reader, docBase);
+        public void setNextReader(AtomicReaderContext context) throws IOException {
+          collector.setNextReader(context);
         }
         
         @Override
@@ -243,10 +241,8 @@ public class ConstantScoreQuery extends 
     }
 
     // this optimization allows out of order scoring as top scorer,
-    // TODO: theoretically this method should not be called because its protected and
-    // this class does not use it, it should be public in Scorer!
     @Override
-    protected boolean score(Collector collector, int max, int firstDocID) throws IOException {
+    public boolean score(Collector collector, int max, int firstDocID) throws IOException {
       if (docIdSetIterator instanceof Scorer) {
         return ((Scorer) docIdSetIterator).score(wrapCollector(collector), max, firstDocID);
       } else {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/DefaultSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/DefaultSimilarity.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/DefaultSimilarity.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/DefaultSimilarity.java Wed Feb  9 09:35:27 2011
@@ -20,7 +20,7 @@ import org.apache.lucene.index.FieldInve
  */
 
 /** Expert: Default scoring implementation. */
-public class DefaultSimilarity extends Similarity {
+public class DefaultSimilarity extends Similarity implements SimilarityProvider {
 
   /** Implemented as
    *  <code>state.getBoost()*lengthNorm(numTerms)</code>, where
@@ -37,17 +37,10 @@ public class DefaultSimilarity extends S
       numTerms = state.getLength() - state.getNumOverlap();
     else
       numTerms = state.getLength();
-    return (state.getBoost() * lengthNorm(field, numTerms));
-  }
-  
-  /** Implemented as <code>1/sqrt(numTerms)</code>. */
-  @Override
-  public float lengthNorm(String fieldName, int numTerms) {
-    return (float)(1.0 / Math.sqrt(numTerms));
+    return state.getBoost() * ((float) (1.0 / Math.sqrt(numTerms)));
   }
   
   /** Implemented as <code>1/sqrt(sumOfSquaredWeights)</code>. */
-  @Override
   public float queryNorm(float sumOfSquaredWeights) {
     return (float)(1.0 / Math.sqrt(sumOfSquaredWeights));
   }
@@ -71,7 +64,6 @@ public class DefaultSimilarity extends S
   }
     
   /** Implemented as <code>overlap / maxOverlap</code>. */
-  @Override
   public float coord(int overlap, int maxOverlap) {
     return overlap / (float)maxOverlap;
   }
@@ -96,4 +88,12 @@ public class DefaultSimilarity extends S
   public boolean getDiscountOverlaps() {
     return discountOverlaps;
   }
+
+  /** 
+   * Returns this default implementation for all fields.
+   * Override this method to customize scoring on a per-field basis.
+   */
+  public Similarity get(String field) {
+    return this;
+  }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java Wed Feb  9 09:35:27 2011
@@ -23,6 +23,7 @@ import java.util.Iterator;
 import java.util.Set;
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
 
 /**
@@ -94,29 +95,26 @@ public class DisjunctionMaxQuery extends
    * change suddenly in the next release.</p>
    */
   protected class DisjunctionMaxWeight extends Weight {
-    /** The Similarity implementation. */
-    protected Similarity similarity;
 
     /** The Weights for our subqueries, in 1-1 correspondence with disjuncts */
     protected ArrayList<Weight> weights = new ArrayList<Weight>();  // 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. */
+    /** Construct the Weight for this Query searched by searcher.  Recursively construct subquery weights. */
     public DisjunctionMaxWeight(IndexSearcher searcher) throws IOException {
-      this.similarity = searcher.getSimilarity();
       for (Query disjunctQuery : disjuncts) {
         weights.add(disjunctQuery.createWeight(searcher));
       }
     }
 
-    /* Return our associated DisjunctionMaxQuery */
+    /** Return our associated DisjunctionMaxQuery */
     @Override
     public Query getQuery() { return DisjunctionMaxQuery.this; }
 
-    /* Return our boost */
+    /** Return our boost */
     @Override
     public float getValue() { return getBoost(); }
 
-    /* Compute the sub of squared weights of us applied to our subqueries.  Used for normalization. */
+    /** Compute the sub of squared weights of us applied to our subqueries.  Used for normalization. */
     @Override
     public float sumOfSquaredWeights() throws IOException {
       float max = 0.0f, sum = 0.0f;
@@ -130,7 +128,7 @@ public class DisjunctionMaxQuery extends
       return (((sum - max) * tieBreakerMultiplier * tieBreakerMultiplier) + max) * boost * boost;
     }
 
-    /* Apply the computed normalization factor to our subqueries */
+    /** Apply the computed normalization factor to our subqueries */
     @Override
     public void normalize(float norm) {
       norm *= getBoost();  // Incorporate our boost
@@ -139,32 +137,31 @@ public class DisjunctionMaxQuery extends
       }
     }
 
-    /* Create the scorer used to score our associated DisjunctionMaxQuery */
+    /** Create the scorer used to score our associated DisjunctionMaxQuery */
     @Override
-    public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder,
-        boolean topScorer) throws IOException {
+    public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
       Scorer[] scorers = new Scorer[weights.size()];
       int idx = 0;
       for (Weight w : weights) {
-        Scorer subScorer = w.scorer(reader, true, false);
+        Scorer subScorer = w.scorer(context, ScorerContext.def());
         if (subScorer != null && subScorer.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
           scorers[idx++] = subScorer;
         }
       }
       if (idx == 0) return null; // all scorers did not have documents
-      DisjunctionMaxScorer result = new DisjunctionMaxScorer(tieBreakerMultiplier, similarity, scorers, idx);
+      DisjunctionMaxScorer result = new DisjunctionMaxScorer(this, tieBreakerMultiplier, scorers, idx);
       return result;
     }
 
-    /* Explain the score we computed for doc */
+    /** Explain the score we computed for doc */
     @Override
-    public Explanation explain(IndexReader reader, int doc) throws IOException {
-      if (disjuncts.size() == 1) return weights.get(0).explain(reader,doc);
+    public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
+      if (disjuncts.size() == 1) return weights.get(0).explain(context,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 (Weight wt : weights) {
-        Explanation e = wt.explain(reader, doc);
+        Explanation e = wt.explain(context, doc);
         if (e.isMatch()) {
           result.setMatch(Boolean.TRUE);
           result.addDetail(e);
@@ -178,7 +175,7 @@ public class DisjunctionMaxQuery extends
     
   }  // end of DisjunctionMaxWeight inner class
 
-  /* Create the Weight used to score us */
+  /** Create the Weight used to score us */
   @Override
   public Weight createWeight(IndexSearcher searcher) throws IOException {
     return new DisjunctionMaxWeight(searcher);

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java Wed Feb  9 09:35:27 2011
@@ -40,22 +40,20 @@ class DisjunctionMaxScorer extends Score
   /**
    * Creates a new instance of DisjunctionMaxScorer
    * 
+   * @param weight
+   *          The Weight to be used.
    * @param tieBreakerMultiplier
    *          Multiplier applied to non-maximum-scoring subqueries for a
    *          document as they are summed into the result.
-   * @param similarity
-   *          -- not used since our definition involves neither coord nor terms
-   *          directly
    * @param subScorers
    *          The sub scorers this Scorer should iterate on
    * @param numScorers
    *          The actual number of scorers to iterate on. Note that the array's
    *          length may be larger than the actual number of scorers.
    */
-  public DisjunctionMaxScorer(float tieBreakerMultiplier,
-      Similarity similarity, Scorer[] subScorers, int numScorers) throws IOException {
-    super(similarity);
-
+  public DisjunctionMaxScorer(Weight weight, float tieBreakerMultiplier,
+      Scorer[] subScorers, int numScorers) throws IOException {
+    super(weight);
     this.tieBreakerMultiplier = tieBreakerMultiplier;
     // The passed subScorers array includes only scorers which have documents
     // (DisjunctionMaxQuery takes care of that), and their nextDoc() was already

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/DisjunctionSumScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/DisjunctionSumScorer.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/DisjunctionSumScorer.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/DisjunctionSumScorer.java Wed Feb  9 09:35:27 2011
@@ -58,6 +58,7 @@ class DisjunctionSumScorer extends Score
   private float currentScore = Float.NaN;
   
   /** Construct a <code>DisjunctionScorer</code>.
+   * @param weight The weight to be used.
    * @param subScorers A collection of at least two subscorers.
    * @param minimumNrMatchers The positive minimum number of subscorers that should
    * match to match this query.
@@ -67,8 +68,8 @@ class DisjunctionSumScorer extends Score
    * <br>When minimumNrMatchers equals the number of subScorers,
    * it more efficient to use <code>ConjunctionScorer</code>.
    */
-  public DisjunctionSumScorer( List<Scorer> subScorers, int minimumNrMatchers) throws IOException {
-    super(null);
+  public DisjunctionSumScorer(Weight weight, List<Scorer> subScorers, int minimumNrMatchers) throws IOException {
+    super(weight);
     
     nrScorers = subScorers.size();
 
@@ -88,8 +89,8 @@ class DisjunctionSumScorer extends Score
   /** Construct a <code>DisjunctionScorer</code>, using one as the minimum number
    * of matching subscorers.
    */
-  public DisjunctionSumScorer(List<Scorer> subScorers) throws IOException {
-    this(subScorers, 1);
+  public DisjunctionSumScorer(Weight weight, List<Scorer> subScorers) throws IOException {
+    this(weight, subScorers, 1);
   }
 
   /** Called the first time nextDoc() or advance() is called to
@@ -123,7 +124,7 @@ class DisjunctionSumScorer extends Score
    * @return true if more matching documents may remain.
    */
   @Override
-  protected boolean score(Collector collector, int max, int firstDocID) throws IOException {
+  public boolean score(Collector collector, int max, int firstDocID) throws IOException {
     // firstDocID is ignored since nextDoc() sets 'currentDoc'
     collector.setScorer(this);
     while (currentDoc < max) {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/DocIdSetIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/DocIdSetIterator.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/DocIdSetIterator.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/DocIdSetIterator.java Wed Feb  9 09:35:27 2011
@@ -78,10 +78,10 @@ public abstract class DocIdSetIterator {
    * 
    * Some implementations are considerably more efficient than that.
    * <p>
-   * <b>NOTE:</b> certain implementations may return a different value (each
-   * time) if called several times in a row with the same target.
+   * <b>NOTE:</b> when <code> target &le; current</code> implementations may opt 
+   * not to advance beyond their current {@link #docID()}.
    * <p>
-   * <b>NOTE:</b> this method may be called with {@value #NO_MORE_DOCS} for
+   * <b>NOTE:</b> this method may be called with {@link #NO_MORE_DOCS} for
    * efficiency by some Scorers. If your implementation cannot efficiently
    * determine that it should exhaust, it is recommended that you check for that
    * value in each call to this method.

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ExactPhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ExactPhraseScorer.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ExactPhraseScorer.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ExactPhraseScorer.java Wed Feb  9 09:35:27 2011
@@ -60,9 +60,12 @@ final class ExactPhraseScorer extends Sc
   private int docID = -1;
   private int freq;
 
+  private final Similarity similarity;
+  
   ExactPhraseScorer(Weight weight, PhraseQuery.PostingsAndFreq[] postings,
                     Similarity similarity, byte[] norms) throws IOException {
-    super(similarity, weight);
+    super(weight);
+    this.similarity = similarity;
     this.norms = norms;
     this.value = weight.getValue();
 
@@ -87,7 +90,7 @@ final class ExactPhraseScorer extends Sc
     }
 
     for (int i = 0; i < SCORE_CACHE_SIZE; i++) {
-      scoreCache[i] = getSimilarity().tf((float) i) * value;
+      scoreCache[i] = similarity.tf((float) i) * value;
     }
   }
 
@@ -207,9 +210,9 @@ final class ExactPhraseScorer extends Sc
     if (freq < SCORE_CACHE_SIZE) {
       raw = scoreCache[freq];
     } else {
-      raw = getSimilarity().tf((float) freq) * value;
+      raw = similarity.tf((float) freq) * value;
     }
-    return norms == null ? raw : raw * getSimilarity().decodeNormValue(norms[docID]); // normalize
+    return norms == null ? raw : raw * similarity.decodeNormValue(norms[docID]); // normalize
   }
 
   private int phraseFreq() throws IOException {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldCacheImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldCacheImpl.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldCacheImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldCacheImpl.java Wed Feb  9 09:35:27 2011
@@ -137,6 +137,13 @@ public class FieldCacheImpl implements F
     public Object getValue() { return value; }
   }
 
+  final static IndexReader.ReaderFinishedListener purgeReader = new IndexReader.ReaderFinishedListener() {
+    // @Override -- not until Java 1.6
+    public void finished(IndexReader reader) {
+      FieldCache.DEFAULT.purge(reader);
+    }
+  };
+
   /** Expert: Internal cache. */
   final static class Cache<T> {
     Cache() {
@@ -171,8 +178,10 @@ public class FieldCacheImpl implements F
       synchronized (readerCache) {
         innerCache = readerCache.get(readerKey);
         if (innerCache == null) {
+          // First time this reader is using FieldCache
           innerCache = new HashMap<Entry<T>,Object>();
           readerCache.put(readerKey, innerCache);
+          reader.addReaderFinishedListener(purgeReader);
           value = null;
         } else {
           value = innerCache.get(key);

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java Wed Feb  9 09:35:27 2011
@@ -19,6 +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.index.MultiFields;
 import org.apache.lucene.util.NumericUtils;
 import org.apache.lucene.util.Bits;
@@ -73,7 +74,7 @@ public abstract class FieldCacheRangeFil
   
   /** This method is implemented for each data type */
   @Override
-  public abstract DocIdSet getDocIdSet(IndexReader reader) throws IOException;
+  public abstract DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException;
 
   /**
    * Creates a string range filter using {@link FieldCache#getTermsIndex}. This works with all
@@ -83,8 +84,8 @@ public abstract class FieldCacheRangeFil
   public static FieldCacheRangeFilter<String> newStringRange(String field, String lowerVal, String upperVal, boolean includeLower, boolean includeUpper) {
     return new FieldCacheRangeFilter<String>(field, null, lowerVal, upperVal, includeLower, includeUpper) {
       @Override
-      public DocIdSet getDocIdSet(IndexReader reader) throws IOException {
-        final FieldCache.DocTermsIndex fcsi = FieldCache.DEFAULT.getTermsIndex(reader, field);
+      public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
+        final FieldCache.DocTermsIndex fcsi = FieldCache.DEFAULT.getTermsIndex(context.reader, field);
         final BytesRef spare = new BytesRef();
         final int lowerPoint = fcsi.binarySearchLookup(lowerVal == null ? null : new BytesRef(lowerVal), spare);
         final int upperPoint = fcsi.binarySearchLookup(upperVal == null ? null : new BytesRef(upperVal), spare);
@@ -124,7 +125,7 @@ public abstract class FieldCacheRangeFil
         
         // for this DocIdSet, we can ignore deleted docs
         // because deleted docs have an order of 0 (null entry in StringIndex)
-        return new FieldCacheDocIdSet(reader, true) {
+        return new FieldCacheDocIdSet(context.reader, true) {
           @Override
           final boolean matchDoc(int doc) {
             final int docOrd = fcsi.getOrd(doc);
@@ -152,7 +153,7 @@ public abstract class FieldCacheRangeFil
   public static FieldCacheRangeFilter<Byte> newByteRange(String field, FieldCache.ByteParser parser, Byte lowerVal, Byte upperVal, boolean includeLower, boolean includeUpper) {
     return new FieldCacheRangeFilter<Byte>(field, parser, lowerVal, upperVal, includeLower, includeUpper) {
       @Override
-      public DocIdSet getDocIdSet(IndexReader reader) throws IOException {
+      public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
         final byte inclusiveLowerPoint, inclusiveUpperPoint;
         if (lowerVal != null) {
           final byte i = lowerVal.byteValue();
@@ -174,9 +175,9 @@ public abstract class FieldCacheRangeFil
         if (inclusiveLowerPoint > inclusiveUpperPoint)
           return DocIdSet.EMPTY_DOCIDSET;
         
-        final byte[] values = FieldCache.DEFAULT.getBytes(reader, field, (FieldCache.ByteParser) parser);
+        final byte[] values = FieldCache.DEFAULT.getBytes(context.reader, field, (FieldCache.ByteParser) parser);
         // we only respect deleted docs if the range contains 0
-        return new FieldCacheDocIdSet(reader, !(inclusiveLowerPoint <= 0 && inclusiveUpperPoint >= 0)) {
+        return new FieldCacheDocIdSet(context.reader, !(inclusiveLowerPoint <= 0 && inclusiveUpperPoint >= 0)) {
           @Override
           boolean matchDoc(int doc) {
             return values[doc] >= inclusiveLowerPoint && values[doc] <= inclusiveUpperPoint;
@@ -203,7 +204,7 @@ public abstract class FieldCacheRangeFil
   public static FieldCacheRangeFilter<Short> newShortRange(String field, FieldCache.ShortParser parser, Short lowerVal, Short upperVal, boolean includeLower, boolean includeUpper) {
     return new FieldCacheRangeFilter<Short>(field, parser, lowerVal, upperVal, includeLower, includeUpper) {
       @Override
-      public DocIdSet getDocIdSet(IndexReader reader) throws IOException {
+      public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
         final short inclusiveLowerPoint, inclusiveUpperPoint;
         if (lowerVal != null) {
           short i = lowerVal.shortValue();
@@ -225,9 +226,9 @@ public abstract class FieldCacheRangeFil
         if (inclusiveLowerPoint > inclusiveUpperPoint)
           return DocIdSet.EMPTY_DOCIDSET;
         
-        final short[] values = FieldCache.DEFAULT.getShorts(reader, field, (FieldCache.ShortParser) parser);
+        final short[] values = FieldCache.DEFAULT.getShorts(context.reader, field, (FieldCache.ShortParser) parser);
         // ignore deleted docs if range doesn't contain 0
-        return new FieldCacheDocIdSet(reader, !(inclusiveLowerPoint <= 0 && inclusiveUpperPoint >= 0)) {
+        return new FieldCacheDocIdSet(context.reader, !(inclusiveLowerPoint <= 0 && inclusiveUpperPoint >= 0)) {
           @Override
           boolean matchDoc(int doc) {
             return values[doc] >= inclusiveLowerPoint && values[doc] <= inclusiveUpperPoint;
@@ -254,7 +255,7 @@ public abstract class FieldCacheRangeFil
   public static FieldCacheRangeFilter<Integer> newIntRange(String field, FieldCache.IntParser parser, Integer lowerVal, Integer upperVal, boolean includeLower, boolean includeUpper) {
     return new FieldCacheRangeFilter<Integer>(field, parser, lowerVal, upperVal, includeLower, includeUpper) {
       @Override
-      public DocIdSet getDocIdSet(IndexReader reader) throws IOException {
+      public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
         final int inclusiveLowerPoint, inclusiveUpperPoint;
         if (lowerVal != null) {
           int i = lowerVal.intValue();
@@ -276,9 +277,9 @@ public abstract class FieldCacheRangeFil
         if (inclusiveLowerPoint > inclusiveUpperPoint)
           return DocIdSet.EMPTY_DOCIDSET;
         
-        final int[] values = FieldCache.DEFAULT.getInts(reader, field, (FieldCache.IntParser) parser);
+        final int[] values = FieldCache.DEFAULT.getInts(context.reader, field, (FieldCache.IntParser) parser);
         // ignore deleted docs if range doesn't contain 0
-        return new FieldCacheDocIdSet(reader, !(inclusiveLowerPoint <= 0 && inclusiveUpperPoint >= 0)) {
+        return new FieldCacheDocIdSet(context.reader, !(inclusiveLowerPoint <= 0 && inclusiveUpperPoint >= 0)) {
           @Override
           boolean matchDoc(int doc) {
             return values[doc] >= inclusiveLowerPoint && values[doc] <= inclusiveUpperPoint;
@@ -305,7 +306,7 @@ public abstract class FieldCacheRangeFil
   public static FieldCacheRangeFilter<Long> newLongRange(String field, FieldCache.LongParser parser, Long lowerVal, Long upperVal, boolean includeLower, boolean includeUpper) {
     return new FieldCacheRangeFilter<Long>(field, parser, lowerVal, upperVal, includeLower, includeUpper) {
       @Override
-      public DocIdSet getDocIdSet(IndexReader reader) throws IOException {
+      public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
         final long inclusiveLowerPoint, inclusiveUpperPoint;
         if (lowerVal != null) {
           long i = lowerVal.longValue();
@@ -327,9 +328,9 @@ public abstract class FieldCacheRangeFil
         if (inclusiveLowerPoint > inclusiveUpperPoint)
           return DocIdSet.EMPTY_DOCIDSET;
         
-        final long[] values = FieldCache.DEFAULT.getLongs(reader, field, (FieldCache.LongParser) parser);
+        final long[] values = FieldCache.DEFAULT.getLongs(context.reader, field, (FieldCache.LongParser) parser);
         // ignore deleted docs if range doesn't contain 0
-        return new FieldCacheDocIdSet(reader, !(inclusiveLowerPoint <= 0L && inclusiveUpperPoint >= 0L)) {
+        return new FieldCacheDocIdSet(context.reader, !(inclusiveLowerPoint <= 0L && inclusiveUpperPoint >= 0L)) {
           @Override
           boolean matchDoc(int doc) {
             return values[doc] >= inclusiveLowerPoint && values[doc] <= inclusiveUpperPoint;
@@ -356,7 +357,7 @@ public abstract class FieldCacheRangeFil
   public static FieldCacheRangeFilter<Float> newFloatRange(String field, FieldCache.FloatParser parser, Float lowerVal, Float upperVal, boolean includeLower, boolean includeUpper) {
     return new FieldCacheRangeFilter<Float>(field, parser, lowerVal, upperVal, includeLower, includeUpper) {
       @Override
-      public DocIdSet getDocIdSet(IndexReader reader) throws IOException {
+      public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
         // we transform the floating point numbers to sortable integers
         // using NumericUtils to easier find the next bigger/lower value
         final float inclusiveLowerPoint, inclusiveUpperPoint;
@@ -382,9 +383,9 @@ public abstract class FieldCacheRangeFil
         if (inclusiveLowerPoint > inclusiveUpperPoint)
           return DocIdSet.EMPTY_DOCIDSET;
         
-        final float[] values = FieldCache.DEFAULT.getFloats(reader, field, (FieldCache.FloatParser) parser);
+        final float[] values = FieldCache.DEFAULT.getFloats(context.reader, field, (FieldCache.FloatParser) parser);
         // ignore deleted docs if range doesn't contain 0
-        return new FieldCacheDocIdSet(reader, !(inclusiveLowerPoint <= 0.0f && inclusiveUpperPoint >= 0.0f)) {
+        return new FieldCacheDocIdSet(context.reader, !(inclusiveLowerPoint <= 0.0f && inclusiveUpperPoint >= 0.0f)) {
           @Override
           boolean matchDoc(int doc) {
             return values[doc] >= inclusiveLowerPoint && values[doc] <= inclusiveUpperPoint;
@@ -411,7 +412,7 @@ public abstract class FieldCacheRangeFil
   public static FieldCacheRangeFilter<Double> newDoubleRange(String field, FieldCache.DoubleParser parser, Double lowerVal, Double upperVal, boolean includeLower, boolean includeUpper) {
     return new FieldCacheRangeFilter<Double>(field, parser, lowerVal, upperVal, includeLower, includeUpper) {
       @Override
-      public DocIdSet getDocIdSet(IndexReader reader) throws IOException {
+      public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
         // we transform the floating point numbers to sortable integers
         // using NumericUtils to easier find the next bigger/lower value
         final double inclusiveLowerPoint, inclusiveUpperPoint;
@@ -437,9 +438,9 @@ public abstract class FieldCacheRangeFil
         if (inclusiveLowerPoint > inclusiveUpperPoint)
           return DocIdSet.EMPTY_DOCIDSET;
         
-        final double[] values = FieldCache.DEFAULT.getDoubles(reader, field, (FieldCache.DoubleParser) parser);
+        final double[] values = FieldCache.DEFAULT.getDoubles(context.reader, field, (FieldCache.DoubleParser) parser);
         // ignore deleted docs if range doesn't contain 0
-        return new FieldCacheDocIdSet(reader, !(inclusiveLowerPoint <= 0.0 && inclusiveUpperPoint >= 0.0)) {
+        return new FieldCacheDocIdSet(context.reader, !(inclusiveLowerPoint <= 0.0 && inclusiveUpperPoint >= 0.0)) {
           @Override
           boolean matchDoc(int doc) {
             return values[doc] >= inclusiveLowerPoint && values[doc] <= inclusiveUpperPoint;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java Wed Feb  9 09:35:27 2011
@@ -21,6 +21,7 @@ import java.io.IOException;
 
 import org.apache.lucene.index.DocsEnum; // javadoc @link
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.util.OpenBitSet;
 import org.apache.lucene.util.BytesRef;
 
@@ -115,8 +116,8 @@ public class FieldCacheTermsFilter exten
   }
 
   @Override
-  public DocIdSet getDocIdSet(IndexReader reader) throws IOException {
-    return new FieldCacheTermsFilterDocIdSet(getFieldCache().getTermsIndex(reader, field));
+  public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
+    return new FieldCacheTermsFilterDocIdSet(getFieldCache().getTermsIndex(context.reader, field));
   }
 
   protected class FieldCacheTermsFilterDocIdSet extends DocIdSet {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldComparator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldComparator.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldComparator.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldComparator.java Wed Feb  9 09:35:27 2011
@@ -21,7 +21,7 @@ import java.io.IOException;
 import java.text.Collator;
 import java.util.Locale;
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.values.DocValues.Source;
 import org.apache.lucene.search.FieldCache.DocTerms;
 import org.apache.lucene.search.FieldCache.DocTermsIndex;
@@ -83,7 +83,7 @@ import org.apache.lucene.util.packed.Pac
  *       priority queue.  The {@link FieldValueHitQueue}
  *       calls this method when a new hit is competitive.
  *
- *  <li> {@link #setNextReader} Invoked
+ *  <li> {@link #setNextReader(IndexReader.AtomicReaderContext)} Invoked
  *       when the search is switching to the next segment.
  *       You may need to update internal state of the
  *       comparator, for example retrieving new values from
@@ -151,18 +151,17 @@ public abstract class FieldComparator {
   public abstract void copy(int slot, int doc) throws IOException;
 
   /**
-   * Set a new Reader. All subsequent docIDs are relative to
+   * Set a new {@link AtomicReaderContext}. All subsequent docIDs are relative to
    * the current reader (you must add docBase if you need to
    * map it to a top-level docID).
    * 
-   * @param reader current reader
-   * @param docBase docBase of this reader 
+   * @param context current reader context
    * @return the comparator to use for this segment; most
    *   comparators can just return "this" to reuse the same
    *   comparator across segments
    * @throws IOException
    */
-  public abstract FieldComparator setNextReader(IndexReader reader, int docBase) throws IOException;
+  public abstract FieldComparator setNextReader(AtomicReaderContext context) throws IOException;
 
   /** Sets the Scorer to use in case a document's score is
    *  needed.
@@ -242,8 +241,8 @@ public abstract class FieldComparator {
     }
 
     @Override
-    public FieldComparator setNextReader(IndexReader reader, int docBase) throws IOException {
-      setup(FieldCache.DEFAULT.getBytes(reader, creator.field, creator));
+    public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
+      setup(FieldCache.DEFAULT.getBytes(context.reader, creator.field, creator));
       docValues = cached.values;
       return this;
     }
@@ -314,8 +313,8 @@ public abstract class FieldComparator {
     }
 
     @Override
-    public FieldComparator setNextReader(IndexReader reader, int docBase) throws IOException {
-      setup(FieldCache.DEFAULT.getDoubles(reader, creator.field, creator));
+    public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
+      setup(FieldCache.DEFAULT.getDoubles(context.reader, creator.field, creator));
       docValues = cached.values;
       return this;
     }
@@ -374,8 +373,8 @@ public abstract class FieldComparator {
     }
 
     @Override
-    public FieldComparator setNextReader(IndexReader reader, int docBase) throws IOException {
-      currentReaderValues = reader.docValues(field).getSource();
+    public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
+      currentReaderValues = context.reader.docValues(field).getSource();
       return this;
     }
     
@@ -447,8 +446,8 @@ public abstract class FieldComparator {
     }
 
     @Override
-    public FieldComparator setNextReader(IndexReader reader, int docBase) throws IOException {
-      setup(FieldCache.DEFAULT.getFloats(reader, creator.field, creator));
+    public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
+      setup(FieldCache.DEFAULT.getFloats(context.reader, creator.field, creator));
       docValues = cached.values;
       return this;
     }
@@ -503,8 +502,8 @@ public abstract class FieldComparator {
     }
 
     @Override
-    public FieldComparator setNextReader(IndexReader reader, int docBase) throws IOException {
-      setup( FieldCache.DEFAULT.getShorts(reader, creator.field, creator));
+    public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
+      setup( FieldCache.DEFAULT.getShorts(context.reader, creator.field, creator));
       docValues = cached.values;
       return this;
     }
@@ -581,8 +580,8 @@ public abstract class FieldComparator {
     }
 
     @Override
-    public FieldComparator setNextReader(IndexReader reader, int docBase) throws IOException {
-      setup(FieldCache.DEFAULT.getInts(reader, creator.field, creator));
+    public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
+      setup(FieldCache.DEFAULT.getInts(context.reader, creator.field, creator));
       docValues = cached.values;
       return this;
     }
@@ -645,8 +644,8 @@ public abstract class FieldComparator {
     }
 
     @Override
-    public FieldComparator setNextReader(IndexReader reader, int docBase) throws IOException {
-      currentReaderValues = reader.docValues(field).getSource();
+    public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
+      currentReaderValues = context.reader.docValues(field).getSource();
       return this;
     }
     
@@ -719,8 +718,8 @@ public abstract class FieldComparator {
     }
 
     @Override
-    public FieldComparator setNextReader(IndexReader reader, int docBase) throws IOException {
-      setup(FieldCache.DEFAULT.getLongs(reader, creator.field, creator));
+    public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
+      setup(FieldCache.DEFAULT.getLongs(context.reader, creator.field, creator));
       docValues = cached.values;
       return this;
     }
@@ -770,7 +769,7 @@ public abstract class FieldComparator {
     }
 
     @Override
-    public FieldComparator setNextReader(IndexReader reader, int docBase) {
+    public FieldComparator setNextReader(AtomicReaderContext context) {
       return this;
     }
     
@@ -822,11 +821,11 @@ public abstract class FieldComparator {
     }
 
     @Override
-    public FieldComparator setNextReader(IndexReader reader, int docBase) {
+    public FieldComparator setNextReader(AtomicReaderContext context) {
       // TODO: can we "map" our docIDs to the current
       // reader? saves having to then subtract on every
       // compare call
-      this.docBase = docBase;
+      this.docBase = context.docBase;
       return this;
     }
     
@@ -903,8 +902,8 @@ public abstract class FieldComparator {
     }
 
     @Override
-    public FieldComparator setNextReader(IndexReader reader, int docBase) throws IOException {
-      currentDocTerms = FieldCache.DEFAULT.getTerms(reader, field);
+    public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
+      currentDocTerms = FieldCache.DEFAULT.getTerms(context.reader, field);
       return this;
     }
     
@@ -998,8 +997,8 @@ public abstract class FieldComparator {
     abstract class PerSegmentComparator extends FieldComparator {
       
       @Override
-      public FieldComparator setNextReader(IndexReader reader, int docBase) throws IOException {
-        return TermOrdValComparator.this.setNextReader(reader, docBase);
+      public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
+        return TermOrdValComparator.this.setNextReader(context);
       }
 
       @Override
@@ -1264,8 +1263,9 @@ public abstract class FieldComparator {
     }
 
     @Override
-    public FieldComparator setNextReader(IndexReader reader, int docBase) throws IOException {
-      termsIndex = FieldCache.DEFAULT.getTermsIndex(reader, field);
+    public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
+      final int docBase = context.docBase;
+      termsIndex = FieldCache.DEFAULT.getTermsIndex(context.reader, field);
       final PackedInts.Reader docToOrd = termsIndex.getDocToOrd();
       FieldComparator perSegComp;
       if (docToOrd instanceof Direct8) {
@@ -1379,8 +1379,8 @@ public abstract class FieldComparator {
     }
 
     @Override
-    public FieldComparator setNextReader(IndexReader reader, int docBase) throws IOException {
-      docTerms = FieldCache.DEFAULT.getTerms(reader, field);
+    public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
+      docTerms = FieldCache.DEFAULT.getTerms(context.reader, field);
       return this;
     }
     

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldDoc.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldDoc.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldDoc.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldDoc.java Wed Feb  9 09:35:27 2011
@@ -43,7 +43,7 @@ public class FieldDoc extends ScoreDoc {
    * Sort object.  Each Object will be either an Integer, Float or String,
    * depending on the type of values in the terms of the original field.
    * @see Sort
-   * @see Searcher#search(Query,Filter,int,Sort)
+   * @see IndexSearcher#search(Query,Filter,int,Sort)
    */
   public Comparable[] fields;