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 [12/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/FieldValueHitQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldValueHitQueue.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldValueHitQueue.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldValueHitQueue.java Wed Feb  9 09:35:27 2011
@@ -28,7 +28,7 @@ import org.apache.lucene.util.PriorityQu
  * 
  * @lucene.experimental
  * @since 2.9
- * @see Searcher#search(Query,Filter,int,Sort)
+ * @see IndexSearcher#search(Query,Filter,int,Sort)
  * @see FieldCache
  */
 public abstract class FieldValueHitQueue extends PriorityQueue<FieldValueHitQueue.Entry> {
@@ -57,9 +57,6 @@ public abstract class FieldValueHitQueue
     public OneComparatorFieldValueHitQueue(SortField[] fields, int size)
         throws IOException {
       super(fields);
-      if (fields.length == 0) {
-        throw new IllegalArgumentException("Sort must contain at least one field");
-      }
 
       SortField field = fields[0];
       setComparator(0,field.getComparator(size, 0));

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Filter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Filter.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Filter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Filter.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; // javadocs
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.util.DocIdBitSet;
 
 /** 
@@ -38,10 +39,13 @@ public abstract class Filter implements 
    * must refer to document IDs for that segment, not for
    * the top-level reader.
    * 
-   * @param reader a {@link IndexReader} instance opened on the index currently
-   *         searched on. The provided reader is always an
-   *         atomic reader, so you can call reader.fields()
-   *         or reader.getDeletedDocs(), for example.
+   * @param context a {@link AtomicReaderContext} instance opened on the index currently
+   *         searched on. Note, it is likely that the provided reader info does not
+   *         represent the whole underlying index i.e. if the index has more than
+   *         one segment the given reader only represents a single segment.
+   *         The provided context is always an atomic context, so you can call 
+   *         {@link IndexReader#fields()} or  {@link IndexReader#getDeletedDocs()}
+   *         on the context's reader, for example.
    *          
    * @return a DocIdSet that provides the documents which should be permitted or
    *         prohibited in search results. <b>NOTE:</b> null can be returned if
@@ -49,5 +53,5 @@ public abstract class Filter implements 
    * 
    * @see DocIdBitSet
    */
-  public abstract DocIdSet getDocIdSet(IndexReader reader) throws IOException;
+  public abstract DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException;
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FilteredQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FilteredQuery.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FilteredQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FilteredQuery.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;
 
@@ -61,7 +62,6 @@ extends Query {
   @Override
   public Weight createWeight(final IndexSearcher searcher) throws IOException {
     final Weight weight = query.createWeight (searcher);
-    final Similarity similarity = query.getSimilarity(searcher);
     return new Weight() {
       private float value;
         
@@ -81,7 +81,7 @@ extends Query {
       }
 
       @Override
-      public Explanation explain (IndexReader ir, int i) throws IOException {
+      public Explanation explain (AtomicReaderContext ir, int i) throws IOException {
         Explanation inner = weight.explain (ir, i);
         if (getBoost()!=1) {
           Explanation preBoost = inner;
@@ -111,13 +111,13 @@ extends Query {
 
       // return a filtering scorer
       @Override
-      public Scorer scorer(IndexReader indexReader, boolean scoreDocsInOrder, boolean topScorer)
+      public Scorer scorer(AtomicReaderContext context, ScorerContext scoreContext)
           throws IOException {
-        final Scorer scorer = weight.scorer(indexReader, true, false);
+        final Scorer scorer = weight.scorer(context, ScorerContext.def());
         if (scorer == null) {
           return null;
         }
-        DocIdSet docIdSet = filter.getDocIdSet(indexReader);
+        DocIdSet docIdSet = filter.getDocIdSet(context);
         if (docIdSet == null) {
           return null;
         }
@@ -126,7 +126,7 @@ extends Query {
           return null;
         }
 
-        return new Scorer(similarity, this) {
+        return new Scorer(this) {
 
           private int doc = -1;
           

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FilteredTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FilteredTermsEnum.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FilteredTermsEnum.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FilteredTermsEnum.java Wed Feb  9 09:35:27 2011
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.Comparator;
 
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.DocsAndPositionsEnum;
@@ -121,10 +122,15 @@ public abstract class FilteredTermsEnum 
   }
     
   @Override
-  public int docFreq() {
+  public int docFreq() throws IOException {
     return tenum.docFreq();
   }
 
+  @Override
+  public long totalTermFreq() throws IOException {
+    return tenum.totalTermFreq();
+  }
+
   /** This enum does not support seeking!
    * @throws UnsupportedOperationException
    */
@@ -155,12 +161,24 @@ public abstract class FilteredTermsEnum 
   public DocsAndPositionsEnum docsAndPositions(Bits bits, DocsAndPositionsEnum reuse) throws IOException {
     return tenum.docsAndPositions(bits, reuse);
   }
-
+  
+  /** This enum does not support seeking!
+   * @throws UnsupportedOperationException
+   */
   @Override
-  public void cacheCurrentTerm() throws IOException {
-    tenum.cacheCurrentTerm();
+  public void seek(BytesRef term, TermState state) throws IOException {
+    throw new UnsupportedOperationException(getClass().getName()+" does not support seeking");
   }
-    
+  
+  /**
+   * Returns the filtered enums term state 
+   */
+  @Override
+  public TermState termState() throws IOException {
+    assert tenum != null;
+    return tenum.termState();
+  }
+
   @SuppressWarnings("fallthrough")
   @Override
   public BytesRef next() throws IOException {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java Wed Feb  9 09:35:27 2011
@@ -20,7 +20,9 @@ package org.apache.lucene.search;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.AutomatonTermsEnum.CompiledAutomaton;
 import org.apache.lucene.util.Attribute;
 import org.apache.lucene.util.AttributeImpl;
 import org.apache.lucene.util.AttributeSource;
@@ -139,18 +141,18 @@ public final class FuzzyTermsEnum extend
    */
   private TermsEnum getAutomatonEnum(int editDistance, BytesRef lastTerm)
       throws IOException {
-    final List<ByteRunAutomaton> runAutomata = initAutomata(editDistance);
+    final List<CompiledAutomaton> runAutomata = initAutomata(editDistance);
     if (editDistance < runAutomata.size()) {
       return new AutomatonFuzzyTermsEnum(runAutomata.subList(0, editDistance + 1)
-          .toArray(new ByteRunAutomaton[editDistance + 1]), lastTerm);
+          .toArray(new CompiledAutomaton[editDistance + 1]), lastTerm);
     } else {
       return null;
     }
   }
 
   /** initialize levenshtein DFAs up to maxDistance, if possible */
-  private List<ByteRunAutomaton> initAutomata(int maxDistance) {
-    final List<ByteRunAutomaton> runAutomata = dfaAtt.automata();
+  private List<CompiledAutomaton> initAutomata(int maxDistance) {
+    final List<CompiledAutomaton> runAutomata = dfaAtt.automata();
     if (runAutomata.size() <= maxDistance && 
         maxDistance <= LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE) {
       LevenshteinAutomata builder = 
@@ -164,7 +166,7 @@ public final class FuzzyTermsEnum extend
             UnicodeUtil.newString(termText, 0, realPrefixLength));
           a = BasicOperations.concatenate(prefix, a);
         }
-        runAutomata.add(new ByteRunAutomaton(a));
+        runAutomata.add(new CompiledAutomaton(a, true));
       }
     }
     return runAutomata;
@@ -240,15 +242,15 @@ public final class FuzzyTermsEnum extend
   
   // proxy all other enum calls to the actual enum
   @Override
-  public int docFreq() {
+  public int docFreq() throws IOException {
     return actualEnum.docFreq();
   }
-  
+
   @Override
-  public void cacheCurrentTerm() throws IOException {
-    actualEnum.cacheCurrentTerm();
+  public long totalTermFreq() throws IOException {
+    return actualEnum.totalTermFreq();
   }
-
+  
   @Override
   public DocsEnum docs(Bits skipDocs, DocsEnum reuse) throws IOException {
     return actualEnum.docs(skipDocs, reuse);
@@ -261,6 +263,16 @@ public final class FuzzyTermsEnum extend
   }
   
   @Override
+  public void seek(BytesRef term, TermState state) throws IOException {
+    actualEnum.seek(term, state);
+  }
+  
+  @Override
+  public TermState termState() throws IOException {
+    return actualEnum.termState();
+  }
+  
+  @Override
   public Comparator<BytesRef> getComparator() throws IOException {
     return actualEnum.getComparator();
   }
@@ -301,10 +313,12 @@ public final class FuzzyTermsEnum extend
     private final BoostAttribute boostAtt =
       attributes().addAttribute(BoostAttribute.class);
     
-    public AutomatonFuzzyTermsEnum(ByteRunAutomaton matchers[], 
+    public AutomatonFuzzyTermsEnum(CompiledAutomaton compiled[], 
         BytesRef lastTerm) throws IOException {
-      super(matchers[matchers.length - 1], tenum, true, null);
-      this.matchers = matchers;
+      super(tenum, compiled[compiled.length - 1]);
+      this.matchers = new ByteRunAutomaton[compiled.length];
+      for (int i = 0; i < compiled.length; i++)
+        this.matchers[i] = compiled[i].runAutomaton;
       this.lastTerm = lastTerm;
       termRef = new BytesRef(term.text());
     }
@@ -552,14 +566,14 @@ public final class FuzzyTermsEnum extend
   
   /** @lucene.internal */
   public static interface LevenshteinAutomataAttribute extends Attribute {
-    public List<ByteRunAutomaton> automata();
+    public List<CompiledAutomaton> automata();
   }
     
   /** @lucene.internal */
   public static final class LevenshteinAutomataAttributeImpl extends AttributeImpl implements LevenshteinAutomataAttribute {
-    private final List<ByteRunAutomaton> automata = new ArrayList<ByteRunAutomaton>();
+    private final List<CompiledAutomaton> automata = new ArrayList<CompiledAutomaton>();
       
-    public List<ByteRunAutomaton> automata() {
+    public List<CompiledAutomaton> automata() {
       return automata;
     }
 
@@ -584,7 +598,7 @@ public final class FuzzyTermsEnum extend
 
     @Override
     public void copyTo(AttributeImpl target) {
-      final List<ByteRunAutomaton> targetAutomata =
+      final List<CompiledAutomaton> targetAutomata =
         ((LevenshteinAutomataAttribute) target).automata();
       targetAutomata.clear();
       targetAutomata.addAll(automata);

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/IndexSearcher.java Wed Feb  9 09:35:27 2011
@@ -18,9 +18,7 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Iterator;
-import java.util.List;
 import java.util.NoSuchElementException;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CompletionService;
@@ -35,8 +33,12 @@ import org.apache.lucene.document.Docume
 import org.apache.lucene.document.FieldSelector;
 import org.apache.lucene.index.CorruptIndexException;
 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.Weight.ScorerContext;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.NIOFSDirectory;    // javadoc
 import org.apache.lucene.util.ReaderUtil;
 import org.apache.lucene.util.ThreadInterruptedException;
 
@@ -56,18 +58,35 @@ import org.apache.lucene.util.ThreadInte
  * use your own (non-Lucene) objects instead.</p>
  */
 public class IndexSearcher {
-  IndexReader reader;
+  final IndexReader reader; // package private for testing!
   private boolean closeReader;
   
   // NOTE: these members might change in incompatible ways
   // in the next release
-  protected final IndexReader[] subReaders;
-  protected final IndexSearcher[] subSearchers;
-  protected final int[] docStarts;
+  protected final ReaderContext readerContext;
+  protected final AtomicReaderContext[] leafContexts;
+  // used with executor - each slice holds a set of leafs executed within one thread
+  protected final LeafSlice[] leafSlices;
+
+  // These are only used for multi-threaded search
   private final ExecutorService executor;
 
-  /** The Similarity implementation used by this searcher. */
-  private Similarity similarity = Similarity.getDefault();
+  // the default SimilarityProvider
+  private static final SimilarityProvider defaultProvider = new DefaultSimilarity();
+  
+  /**
+   * Expert: returns a default SimilarityProvider instance.
+   * In general, this method is only called to initialize searchers and writers.
+   * User code and query implementations should respect
+   * {@link IndexSearcher#getSimilarityProvider()}.
+   * @lucene.internal
+   */
+  public static SimilarityProvider getDefaultSimilarityProvider() {
+    return defaultProvider;
+  }
+  
+  /** The SimilarityProvider implementation used by this searcher. */
+  private SimilarityProvider similarityProvider = defaultProvider;
 
   /** Creates a searcher searching the index in the named
    *  directory, with readOnly=true
@@ -114,83 +133,71 @@ public class IndexSearcher {
     this(r, false, executor);
   }
 
-  /** Expert: directly specify the reader, subReaders and
-   *  their docID starts.
-   * 
-   * @lucene.experimental */
-  public IndexSearcher(IndexReader reader, IndexReader[] subReaders, int[] docStarts) {
-    this.reader = reader;
-    this.subReaders = subReaders;
-    this.docStarts = docStarts;
-    subSearchers = new IndexSearcher[subReaders.length];
-    for(int i=0;i<subReaders.length;i++) {
-      subSearchers[i] = new IndexSearcher(subReaders[i]);
-    }
-    closeReader = false;
-    executor = null;
-  }
-  
-  /** Expert: directly specify the reader, subReaders and
-   *  their docID starts, and an ExecutorService.  In this
-   *  case, each segment will be separately searched using the
-   *  ExecutorService.  IndexSearcher will not
-   *  shutdown/awaitTermination this ExecutorService on
-   *  close; you must do so, eventually, on your own.  NOTE:
-   *  if you are using {@link NIOFSDirectory}, do not use
-   *  the shutdownNow method of ExecutorService as this uses
-   *  Thread.interrupt under-the-hood which can silently
-   *  close file descriptors (see <a
-   *  href="https://issues.apache.org/jira/browse/LUCENE-2239">LUCENE-2239</a>).
-   * 
-   * @lucene.experimental */
-  public IndexSearcher(IndexReader reader, IndexReader[] subReaders, int[] docStarts, ExecutorService executor) {
-    this.reader = reader;
-    this.subReaders = subReaders;
-    this.docStarts = docStarts;
-    subSearchers = new IndexSearcher[subReaders.length];
-    for(int i=0;i<subReaders.length;i++) {
-      subSearchers[i] = new IndexSearcher(subReaders[i]);
-    }
-    closeReader = false;
-    this.executor = executor;
+  /**
+   * Creates a searcher searching the provided top-level {@link ReaderContext}.
+   * <p>
+   * Given a non-<code>null</code> {@link ExecutorService} this method runs
+   * searches for each segment separately, using the provided ExecutorService.
+   * IndexSearcher will not shutdown/awaitTermination this ExecutorService on
+   * close; you must do so, eventually, on your own. NOTE: if you are using
+   * {@link NIOFSDirectory}, do not use the shutdownNow method of
+   * ExecutorService as this uses Thread.interrupt under-the-hood which can
+   * silently close file descriptors (see <a
+   * href="https://issues.apache.org/jira/browse/LUCENE-2239">LUCENE-2239</a>).
+   * 
+   * @see ReaderContext
+   * @see IndexReader#getTopReaderContext()
+   * @lucene.experimental
+   */
+  public IndexSearcher(ReaderContext context, ExecutorService executor) {
+    this(context, false, executor);
+  }
+
+  /**
+   * Creates a searcher searching the provided top-level {@link ReaderContext}.
+   *
+   * @see ReaderContext
+   * @see IndexReader#getTopReaderContext()
+   * @lucene.experimental
+   */
+  public IndexSearcher(ReaderContext context) {
+    this(context, (ExecutorService) null);
+  }
+  
+  // convenience ctor for other IR based ctors
+  private IndexSearcher(IndexReader reader, boolean closeReader, ExecutorService executor) {
+    this(reader.getTopReaderContext(), closeReader, executor);
   }
 
-  private IndexSearcher(IndexReader r, boolean closeReader, ExecutorService executor) {
-    reader = r;
+  private IndexSearcher(ReaderContext context, boolean closeReader, ExecutorService executor) {
+    assert context.isTopLevel: "IndexSearcher's ReaderContext must be topLevel for reader" + context.reader;
+    reader = context.reader;
     this.executor = executor;
     this.closeReader = closeReader;
-
-    List<IndexReader> subReadersList = new ArrayList<IndexReader>();
-    gatherSubReaders(subReadersList, reader);
-    subReaders = subReadersList.toArray(new IndexReader[subReadersList.size()]);
-    docStarts = new int[subReaders.length];
-    subSearchers = new IndexSearcher[subReaders.length];
-    int maxDoc = 0;
-    for (int i = 0; i < subReaders.length; i++) {
-      docStarts[i] = maxDoc;
-      maxDoc += subReaders[i].maxDoc();
-      if (subReaders[i] == r) {
-        subSearchers[i] = this;
-      } else {
-        subSearchers[i] = new IndexSearcher(subReaders[i]);
-      }
-    }
+    this.readerContext = context;
+    leafContexts = ReaderUtil.leaves(context);
+    this.leafSlices = executor == null ? null : slices(leafContexts);
   }
-
-  protected void gatherSubReaders(List<IndexReader> allSubReaders, IndexReader r) {
-    ReaderUtil.gatherSubReaders(allSubReaders, r);
+  
+  /**
+   * Expert: Creates an array of leaf slices each holding a subset of the given leaves.
+   * Each {@link LeafSlice} is executed in a single thread. By default there
+   * will be one {@link LeafSlice} per leaf ({@link AtomicReaderContext}).
+   */
+  protected LeafSlice[] slices(AtomicReaderContext...leaves) {
+    LeafSlice[] slices = new LeafSlice[leaves.length];
+    for (int i = 0; i < slices.length; i++) {
+      slices[i] = new LeafSlice(leaves[i]);
+    }
+    return slices;
   }
 
+  
   /** Return the {@link IndexReader} this searches. */
   public IndexReader getIndexReader() {
     return reader;
   }
 
-  /** Returns the atomic subReaders used by this searcher. */
-  public IndexReader[] getSubReaders() {
-    return subReaders;
-  }
-
   /** Expert: Returns one greater than the largest possible document number.
    * 
    * @see org.apache.lucene.index.IndexReader#maxDoc()
@@ -205,11 +212,11 @@ public class IndexSearcher {
       return reader.docFreq(term);
     } else {
       final ExecutionHelper<Integer> runner = new ExecutionHelper<Integer>(executor);
-      for(int i = 0; i < subReaders.length; i++) {
-        final IndexSearcher searchable = subSearchers[i];
+      for(int i = 0; i < leafContexts.length; i++) {
+        final IndexReader leaf = leafContexts[i].reader;
         runner.submit(new Callable<Integer>() {
             public Integer call() throws IOException {
-              return Integer.valueOf(searchable.docFreq(term));
+              return Integer.valueOf(leaf.docFreq(term));
             }
           });
       }
@@ -231,16 +238,15 @@ public class IndexSearcher {
     return reader.document(docID, fieldSelector);
   }
   
-  /** Expert: Set the Similarity implementation used by this Searcher.
+  /** Expert: Set the SimilarityProvider implementation used by this Searcher.
    *
-   * @see Similarity#setDefault(Similarity)
    */
-  public void setSimilarity(Similarity similarity) {
-    this.similarity = similarity;
+  public void setSimilarityProvider(SimilarityProvider similarityProvider) {
+    this.similarityProvider = similarityProvider;
   }
 
-  public Similarity getSimilarity() {
-    return similarity;
+  public SimilarityProvider getSimilarityProvider() {
+    return similarityProvider;
   }
 
   /**
@@ -284,7 +290,7 @@ public class IndexSearcher {
    *
    * <p>Applications should only use this if they need <i>all</i> of the
    * matching documents.  The high-level search API ({@link
-   * Searcher#search(Query, Filter, int)}) is usually more efficient, as it skips
+   * IndexSearcher#search(Query, Filter, int)}) is usually more efficient, as it skips
    * non-high-scoring hits.
    *
    * @param query to match documents
@@ -294,7 +300,7 @@ public class IndexSearcher {
    */
   public void search(Query query, Filter filter, Collector results)
     throws IOException {
-    search(createWeight(query), filter, results);
+    search(leafContexts, createWeight(query), filter, results);
   }
 
   /** Lower-level search API.
@@ -303,7 +309,7 @@ public class IndexSearcher {
   *
   * <p>Applications should only use this if they need <i>all</i> of the
   * matching documents.  The high-level search API ({@link
-  * Searcher#search(Query, int)}) is usually more efficient, as it skips
+  * IndexSearcher#search(Query, int)}) is usually more efficient, as it skips
   * non-high-scoring hits.
   * <p>Note: The <code>score</code> passed to this method is a raw score.
   * In other words, the score will not necessarily be a float whose value is
@@ -312,7 +318,7 @@ public class IndexSearcher {
   */
   public void search(Query query, Collector results)
     throws IOException {
-    search(createWeight(query), null, results);
+    search(leafContexts, createWeight(query), null, results);
   }
   
   /** Search implementation with arbitrary sorting.  Finds
@@ -347,37 +353,30 @@ public class IndexSearcher {
   /** Expert: Low-level search implementation.  Finds the top <code>n</code>
    * hits for <code>query</code>, applying <code>filter</code> if non-null.
    *
-   * <p>Applications should usually call {@link Searcher#search(Query,int)} or
-   * {@link Searcher#search(Query,Filter,int)} instead.
+   * <p>Applications should usually call {@link IndexSearcher#search(Query,int)} or
+   * {@link IndexSearcher#search(Query,Filter,int)} instead.
    * @throws BooleanQuery.TooManyClauses
    */
   protected TopDocs search(Weight weight, Filter filter, int nDocs) throws IOException {
-
     if (executor == null) {
-      // single thread
-      int limit = reader.maxDoc();
-      if (limit == 0) {
-        limit = 1;
-      }
-      nDocs = Math.min(nDocs, limit);
-      TopScoreDocCollector collector = TopScoreDocCollector.create(nDocs, !weight.scoresDocsOutOfOrder());
-      search(weight, filter, collector);
-      return collector.topDocs();
+      return search(leafContexts, weight, filter, nDocs);
     } else {
       final HitQueue hq = new HitQueue(nDocs, false);
       final Lock lock = new ReentrantLock();
       final ExecutionHelper<TopDocs> runner = new ExecutionHelper<TopDocs>(executor);
     
-      for (int i = 0; i < subReaders.length; i++) { // search each sub
+      for (int i = 0; i < leafSlices.length; i++) { // search each sub
         runner.submit(
-                      new MultiSearcherCallableNoSort(lock, subSearchers[i], weight, filter, nDocs, hq, i, docStarts));
+                      new SearcherCallableNoSort(lock, this, leafSlices[i], weight, filter, nDocs, hq));
       }
 
       int totalHits = 0;
       float maxScore = Float.NEGATIVE_INFINITY;
       for (final TopDocs topDocs : runner) {
-        totalHits += topDocs.totalHits;
-        maxScore = Math.max(maxScore, topDocs.getMaxScore());
+        if(topDocs.totalHits != 0) {
+          totalHits += topDocs.totalHits;
+          maxScore = Math.max(maxScore, topDocs.getMaxScore());
+        }
       }
 
       final ScoreDoc[] scoreDocs = new ScoreDoc[hq.size()];
@@ -388,13 +387,32 @@ public class IndexSearcher {
     }
   }
 
+  /** Expert: Low-level search implementation.  Finds the top <code>n</code>
+   * hits for <code>query</code>, using the given leaf readers applying <code>filter</code> if non-null.
+   *
+   * <p>Applications should usually call {@link IndexSearcher#search(Query,int)} or
+   * {@link IndexSearcher#search(Query,Filter,int)} instead.
+   * @throws BooleanQuery.TooManyClauses
+   */
+  protected TopDocs search(AtomicReaderContext[] leaves, Weight weight, Filter filter, int nDocs) throws IOException {
+    // single thread
+    int limit = reader.maxDoc();
+    if (limit == 0) {
+      limit = 1;
+    }
+    nDocs = Math.min(nDocs, limit);
+    TopScoreDocCollector collector = TopScoreDocCollector.create(nDocs, !weight.scoresDocsOutOfOrder());
+    search(leaves, weight, filter, collector);
+    return collector.topDocs();
+  }
+
   /** Expert: Low-level search implementation with arbitrary sorting.  Finds
    * the top <code>n</code> hits for <code>query</code>, applying
    * <code>filter</code> if non-null, and sorting the hits by the criteria in
    * <code>sort</code>.
    *
    * <p>Applications should usually call {@link
-   * Searcher#search(Query,Filter,int,Sort)} instead.
+   * IndexSearcher#search(Query,Filter,int,Sort)} instead.
    * 
    * @throws BooleanQuery.TooManyClauses
    */
@@ -419,33 +437,26 @@ public class IndexSearcher {
       throws IOException {
 
     if (sort == null) throw new NullPointerException();
-
+    
     if (executor == null) {
-      // single thread
-      int limit = reader.maxDoc();
-      if (limit == 0) {
-        limit = 1;
-      }
-      nDocs = Math.min(nDocs, limit);
-
-      TopFieldCollector collector = TopFieldCollector.create(sort, nDocs,
-                                                             fillFields, fieldSortDoTrackScores, fieldSortDoMaxScore, !weight.scoresDocsOutOfOrder());
-      search(weight, filter, collector);
-      return (TopFieldDocs) collector.topDocs();
+      // use all leaves here!
+      return search (leafContexts, weight, filter, nDocs, sort, fillFields);
     } else {
       // TODO: make this respect fillFields
       final FieldDocSortedHitQueue hq = new FieldDocSortedHitQueue(nDocs);
       final Lock lock = new ReentrantLock();
       final ExecutionHelper<TopFieldDocs> runner = new ExecutionHelper<TopFieldDocs>(executor);
-      for (int i = 0; i < subReaders.length; i++) { // search each sub
+      for (int i = 0; i < leafSlices.length; i++) { // search each leaf slice
         runner.submit(
-                      new MultiSearcherCallableWithSort(lock, subSearchers[i], weight, filter, nDocs, hq, sort, i, docStarts));
+                      new SearcherCallableWithSort(lock, this, leafSlices[i], weight, filter, nDocs, hq, sort));
       }
       int totalHits = 0;
       float maxScore = Float.NEGATIVE_INFINITY;
       for (final TopFieldDocs topFieldDocs : runner) {
-        totalHits += topFieldDocs.totalHits;
-        maxScore = Math.max(maxScore, topFieldDocs.getMaxScore());
+        if (topFieldDocs.totalHits != 0) {
+          totalHits += topFieldDocs.totalHits;
+          maxScore = Math.max(maxScore, topFieldDocs.getMaxScore());
+        }
       }
       final ScoreDoc[] scoreDocs = new ScoreDoc[hq.size()];
       for (int i = hq.size() - 1; i >= 0; i--) // put docs in array
@@ -454,6 +465,33 @@ public class IndexSearcher {
       return new TopFieldDocs(totalHits, scoreDocs, hq.getFields(), maxScore);
     }
   }
+  
+  
+  /**
+   * Just like {@link #search(Weight, Filter, int, Sort)}, but you choose
+   * whether or not the fields in the returned {@link FieldDoc} instances should
+   * be set by specifying fillFields.
+   *
+   * <p>NOTE: this does not compute scores by default.  If you
+   * need scores, create a {@link TopFieldCollector}
+   * instance by calling {@link TopFieldCollector#create} and
+   * then pass that to {@link #search(Weight, Filter,
+   * Collector)}.</p>
+   */
+  protected TopFieldDocs search(AtomicReaderContext[] leaves, Weight weight, Filter filter, int nDocs,
+      Sort sort, boolean fillFields) throws IOException {
+    // single thread
+    int limit = reader.maxDoc();
+    if (limit == 0) {
+      limit = 1;
+    }
+    nDocs = Math.min(nDocs, limit);
+
+    TopFieldCollector collector = TopFieldCollector.create(sort, nDocs,
+                                                           fillFields, fieldSortDoTrackScores, fieldSortDoMaxScore, !weight.scoresDocsOutOfOrder());
+    search(leaves, weight, filter, collector);
+    return (TopFieldDocs) collector.topDocs();
+  }
 
   /**
    * Lower-level search API.
@@ -464,9 +502,15 @@ public class IndexSearcher {
    * 
    * <p>
    * Applications should only use this if they need <i>all</i> of the matching
-   * documents. The high-level search API ({@link Searcher#search(Query,int)}) is
+   * documents. The high-level search API ({@link IndexSearcher#search(Query,int)}) is
    * usually more efficient, as it skips non-high-scoring hits.
    * 
+   * <p>
+   * NOTE: this method executes the searches on all given leaves exclusively.
+   * To search across all the searchers leaves use {@link #leafContexts}.
+   * 
+   * @param leaves 
+   *          the searchers leaves to execute the searches on
    * @param weight
    *          to match documents
    * @param filter
@@ -475,35 +519,36 @@ public class IndexSearcher {
    *          to receive hits
    * @throws BooleanQuery.TooManyClauses
    */
-  protected void search(Weight weight, Filter filter, Collector collector)
+  protected void search(AtomicReaderContext[] leaves, Weight weight, Filter filter, Collector collector)
       throws IOException {
 
     // TODO: should we make this
     // threaded...?  the Collector could be sync'd?
-
+    ScorerContext scorerContext =  ScorerContext.def().scoreDocsInOrder(true).topScorer(true);
     // always use single thread:
     if (filter == null) {
-      for (int i = 0; i < subReaders.length; i++) { // search each subreader
-        collector.setNextReader(subReaders[i], docStarts[i]);
-        Scorer scorer = weight.scorer(subReaders[i], !collector.acceptsDocsOutOfOrder(), true);
+      for (int i = 0; i < leaves.length; i++) { // search each subreader
+        collector.setNextReader(leaves[i]);
+        scorerContext = scorerContext.scoreDocsInOrder(!collector.acceptsDocsOutOfOrder());
+        Scorer scorer = weight.scorer(leaves[i], scorerContext);
         if (scorer != null) {
           scorer.score(collector);
         }
       }
     } else {
-      for (int i = 0; i < subReaders.length; i++) { // search each subreader
-        collector.setNextReader(subReaders[i], docStarts[i]);
-        searchWithFilter(subReaders[i], weight, filter, collector);
+      for (int i = 0; i < leaves.length; i++) { // search each subreader
+        collector.setNextReader(leaves[i]);
+        searchWithFilter(leaves[i], weight, filter, collector);
       }
     }
   }
 
-  private void searchWithFilter(IndexReader reader, Weight weight,
+  private void searchWithFilter(AtomicReaderContext context, Weight weight,
       final Filter filter, final Collector collector) throws IOException {
 
     assert filter != null;
     
-    Scorer scorer = weight.scorer(reader, true, false);
+    Scorer scorer = weight.scorer(context, ScorerContext.def());
     if (scorer == null) {
       return;
     }
@@ -512,7 +557,7 @@ public class IndexSearcher {
     assert docID == -1 || docID == DocIdSetIterator.NO_MORE_DOCS;
 
     // CHECKME: use ConjunctionScorer here?
-    DocIdSet filterDocIdSet = filter.getDocIdSet(reader);
+    DocIdSet filterDocIdSet = filter.getDocIdSet(context);
     if (filterDocIdSet == null) {
       // this means the filter does not accept any documents.
       return;
@@ -576,14 +621,14 @@ public class IndexSearcher {
    * and, for good performance, should not be displayed with every hit.
    * Computing an explanation is as expensive as executing the query over the
    * entire index.
-   * <p>Applications should call {@link Searcher#explain(Query, int)}.
+   * <p>Applications should call {@link IndexSearcher#explain(Query, int)}.
    * @throws BooleanQuery.TooManyClauses
    */
   protected Explanation explain(Weight weight, int doc) throws IOException {
-    int n = ReaderUtil.subIndex(doc, docStarts);
-    int deBasedDoc = doc - docStarts[n];
+    int n = ReaderUtil.subIndex(doc, leafContexts);
+    int deBasedDoc = doc - leafContexts[n].docBase;
     
-    return weight.explain(subReaders[n], deBasedDoc);
+    return weight.explain(leafContexts[n], deBasedDoc);
   }
 
   private boolean fieldSortDoTrackScores;
@@ -614,39 +659,44 @@ public class IndexSearcher {
     return query.weight(this);
   }
 
+  /**
+   * Returns this searchers the top-level {@link ReaderContext}.
+   * @see IndexReader#getTopReaderContext()
+   */
+  /* sugar for #getReader().getTopReaderContext() */
+  public ReaderContext getTopReaderContext() {
+    return readerContext;
+  }
 
   /**
    * A thread subclass for searching a single searchable 
    */
-  private static final class MultiSearcherCallableNoSort implements Callable<TopDocs> {
+  private static final class SearcherCallableNoSort implements Callable<TopDocs> {
 
     private final Lock lock;
-    private final IndexSearcher searchable;
+    private final IndexSearcher searcher;
     private final Weight weight;
     private final Filter filter;
     private final int nDocs;
-    private final int i;
     private final HitQueue hq;
-    private final int[] starts;
+    private final LeafSlice slice;
 
-    public MultiSearcherCallableNoSort(Lock lock, IndexSearcher searchable, Weight weight,
-        Filter filter, int nDocs, HitQueue hq, int i, int[] starts) {
+    public SearcherCallableNoSort(Lock lock, IndexSearcher searcher, LeafSlice slice,  Weight weight,
+        Filter filter, int nDocs, HitQueue hq) {
       this.lock = lock;
-      this.searchable = searchable;
+      this.searcher = searcher;
       this.weight = weight;
       this.filter = filter;
       this.nDocs = nDocs;
       this.hq = hq;
-      this.i = i;
-      this.starts = starts;
+      this.slice = slice;
     }
 
     public TopDocs call() throws IOException {
-      final TopDocs docs = searchable.search (weight, filter, nDocs);
+      final TopDocs docs = searcher.search (slice.leaves, weight, filter, nDocs);
       final ScoreDoc[] scoreDocs = docs.scoreDocs;
       for (int j = 0; j < scoreDocs.length; j++) { // merge scoreDocs into hq
         final ScoreDoc scoreDoc = scoreDocs[j];
-        scoreDoc.doc += starts[i]; // convert doc 
         //it would be so nice if we had a thread-safe insert 
         lock.lock();
         try {
@@ -664,47 +714,31 @@ public class IndexSearcher {
   /**
    * A thread subclass for searching a single searchable 
    */
-  private static final class MultiSearcherCallableWithSort implements Callable<TopFieldDocs> {
+  private static final class SearcherCallableWithSort implements Callable<TopFieldDocs> {
 
     private final Lock lock;
-    private final IndexSearcher searchable;
+    private final IndexSearcher searcher;
     private final Weight weight;
     private final Filter filter;
     private final int nDocs;
-    private final int i;
     private final FieldDocSortedHitQueue hq;
-    private final int[] starts;
     private final Sort sort;
+    private final LeafSlice slice;
 
-    public MultiSearcherCallableWithSort(Lock lock, IndexSearcher searchable, Weight weight,
-        Filter filter, int nDocs, FieldDocSortedHitQueue hq, Sort sort, int i, int[] starts) {
+    public SearcherCallableWithSort(Lock lock, IndexSearcher searcher, LeafSlice slice, Weight weight,
+        Filter filter, int nDocs, FieldDocSortedHitQueue hq, Sort sort) {
       this.lock = lock;
-      this.searchable = searchable;
+      this.searcher = searcher;
       this.weight = weight;
       this.filter = filter;
       this.nDocs = nDocs;
       this.hq = hq;
-      this.i = i;
-      this.starts = starts;
       this.sort = sort;
+      this.slice = slice;
     }
 
     public TopFieldDocs call() throws IOException {
-      final TopFieldDocs docs = searchable.search (weight, filter, nDocs, sort);
-      // If one of the Sort fields is FIELD_DOC, need to fix its values, so that
-      // it will break ties by doc Id properly. Otherwise, it will compare to
-      // 'relative' doc Ids, that belong to two different searchables.
-      for (int j = 0; j < docs.fields.length; j++) {
-        if (docs.fields[j].getType() == SortField.DOC) {
-          // iterate over the score docs and change their fields value
-          for (int j2 = 0; j2 < docs.scoreDocs.length; j2++) {
-            FieldDoc fd = (FieldDoc) docs.scoreDocs[j2];
-            fd.fields[j] = Integer.valueOf(((Integer) fd.fields[j]).intValue() + starts[i]);
-          }
-          break;
-        }
-      }
-
+      final TopFieldDocs docs = searcher.search (slice.leaves, weight, filter, nDocs, sort, true);
       lock.lock();
       try {
         hq.setFields(docs.fields);
@@ -715,7 +749,6 @@ public class IndexSearcher {
       final ScoreDoc[] scoreDocs = docs.scoreDocs;
       for (int j = 0; j < scoreDocs.length; j++) { // merge scoreDocs into hq
         final FieldDoc fieldDoc = (FieldDoc) scoreDocs[j];
-        fieldDoc.doc += starts[i]; // convert doc 
         //it would be so nice if we had a thread-safe insert 
         lock.lock();
         try {
@@ -772,8 +805,22 @@ public class IndexSearcher {
     }
 
     public Iterator<T> iterator() {
-      // use the shortcut here - this is only used in a privat context
+      // use the shortcut here - this is only used in a private context
       return this;
     }
   }
+
+  /**
+   * A class holding a subset of the {@link IndexSearcher}s leaf contexts to be
+   * executed within a single thread.
+   * 
+   * @lucene.experimental
+   */
+  public static class LeafSlice {
+    final AtomicReaderContext[] leaves;
+    
+    public LeafSlice(AtomicReaderContext...leaves) {
+      this.leaves = leaves;
+    }
+  }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MatchAllDocsQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MatchAllDocsQuery.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MatchAllDocsQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MatchAllDocsQuery.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.util.Bits;
@@ -50,10 +51,12 @@ public class MatchAllDocsQuery extends Q
     private int doc = -1;
     private final int maxDoc;
     private final Bits delDocs;
+    private final Similarity similarity;
     
     MatchAllScorer(IndexReader reader, Similarity similarity, Weight w,
         byte[] norms) throws IOException {
-      super(similarity,w);
+      super(w);
+      this.similarity = similarity;
       delDocs = reader.getDeletedDocs();
       score = w.getValue();
       maxDoc = reader.maxDoc();
@@ -79,7 +82,7 @@ public class MatchAllDocsQuery extends Q
     
     @Override
     public float score() {
-      return norms == null ? score : score * getSimilarity().decodeNormValue(norms[docID()]);
+      return norms == null ? score : score * similarity.decodeNormValue(norms[docID()]);
     }
 
     @Override
@@ -95,7 +98,7 @@ public class MatchAllDocsQuery extends Q
     private float queryNorm;
 
     public MatchAllDocsWeight(IndexSearcher searcher) {
-      this.similarity = searcher.getSimilarity();
+      this.similarity = normsField == null ? null : searcher.getSimilarityProvider().get(normsField);
     }
 
     @Override
@@ -126,13 +129,13 @@ public class MatchAllDocsQuery extends Q
     }
 
     @Override
-    public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
-      return new MatchAllScorer(reader, similarity, this,
-          normsField != null ? reader.norms(normsField) : null);
+    public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
+      return new MatchAllScorer(context.reader, similarity, this,
+          normsField != null ? context.reader.norms(normsField) : null);
     }
 
     @Override
-    public Explanation explain(IndexReader reader, int doc) {
+    public Explanation explain(AtomicReaderContext context, int doc) {
       // explain query weight
       Explanation queryExpl = new ComplexExplanation
         (true, getValue(), "MatchAllDocsQuery, product of:");

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MaxNonCompetitiveBoostAttribute.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MaxNonCompetitiveBoostAttribute.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MaxNonCompetitiveBoostAttribute.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MaxNonCompetitiveBoostAttribute.java Wed Feb  9 09:35:27 2011
@@ -22,13 +22,13 @@ import org.apache.lucene.util.AttributeS
 import org.apache.lucene.util.BytesRef;
 
 /** Add this {@link Attribute} to a fresh {@link AttributeSource} before calling
- * {@link MultiTermQuery#getTermsEnum(IndexReader,AttributeSource)}.
+ * {@link MultiTermQuery#getTermsEnum(Terms,AttributeSource)}.
  * {@link FuzzyQuery} is using this to control its internal behaviour
  * to only return competitive terms.
  * <p><b>Please note:</b> This attribute is intended to be added by the {@link MultiTermQuery.RewriteMethod}
  * to an empty {@link AttributeSource} that is shared for all segments
  * during query rewrite. This attribute source is passed to all segment enums
- * on {@link MultiTermQuery#getTermsEnum(IndexReader,AttributeSource)}.
+ * on {@link MultiTermQuery#getTermsEnum(Terms,AttributeSource)}.
  * {@link TopTermsRewrite} uses this attribute to
  * inform all enums about the current boost, that is not competitive.
  * @lucene.internal

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MaxNonCompetitiveBoostAttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MaxNonCompetitiveBoostAttributeImpl.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MaxNonCompetitiveBoostAttributeImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MaxNonCompetitiveBoostAttributeImpl.java Wed Feb  9 09:35:27 2011
@@ -48,25 +48,6 @@ public final class MaxNonCompetitiveBoos
     maxNonCompetitiveBoost = Float.NEGATIVE_INFINITY;
     competitiveTerm = null;
   }
-
-  @Override
-  public boolean equals(Object other) {
-    if (this == other)
-      return true;
-    if (other instanceof MaxNonCompetitiveBoostAttributeImpl) {
-      final MaxNonCompetitiveBoostAttributeImpl o = (MaxNonCompetitiveBoostAttributeImpl) other;
-      return (o.maxNonCompetitiveBoost == maxNonCompetitiveBoost)
-        && (o.competitiveTerm == null ? competitiveTerm == null : o.competitiveTerm.equals(competitiveTerm));
-    }
-    return false;
-  }
-
-  @Override
-  public int hashCode() {
-    int hash = Float.floatToIntBits(maxNonCompetitiveBoost);
-    if (competitiveTerm != null) hash = 31 * hash + competitiveTerm.hashCode();
-    return hash;
-  }
   
   @Override
   public void copyTo(AttributeImpl target) {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MultiCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MultiCollector.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MultiCollector.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MultiCollector.java Wed Feb  9 09:35:27 2011
@@ -19,14 +19,14 @@ 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.Collector;
 import org.apache.lucene.search.Scorer;
 
 /**
  * A {@link Collector} which allows running a search with several
  * {@link Collector}s. It offers a static {@link #wrap} method which accepts a
- * list of collectots and wraps them with {@link MultiCollector}, while
+ * list of collectors and wraps them with {@link MultiCollector}, while
  * filtering out the <code>null</code> null ones.
  */
 public class MultiCollector extends Collector {
@@ -108,9 +108,9 @@ public class MultiCollector extends Coll
   }
 
   @Override
-  public void setNextReader(IndexReader reader, int o) throws IOException {
+  public void setNextReader(AtomicReaderContext context) throws IOException {
     for (Collector c : collectors) {
-      c.setNextReader(reader, o);
+      c.setNextReader(context);
     }
   }
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java Wed Feb  9 09:35:27 2011
@@ -21,9 +21,11 @@ import java.io.IOException;
 import java.util.*;
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.search.Explanation.IDFExplanation;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.ToStringUtils;
@@ -130,21 +132,24 @@ public class MultiPhraseQuery extends Qu
   private class MultiPhraseWeight extends Weight {
     private Similarity similarity;
     private float value;
+    private final IDFExplanation idfExp;
     private float idf;
     private float queryNorm;
     private float queryWeight;
 
     public MultiPhraseWeight(IndexSearcher searcher)
       throws IOException {
-      this.similarity = getSimilarity(searcher);
+      this.similarity = searcher.getSimilarityProvider().get(field);
 
       // compute idf
-      final int maxDoc = searcher.maxDoc();
+      ArrayList<Term> allTerms = new ArrayList<Term>();
       for(final Term[] terms: termArrays) {
         for (Term term: terms) {
-          idf += this.similarity.idf(searcher.docFreq(term), maxDoc);
+          allTerms.add(term);
         }
       }
+      idfExp = similarity.idfExplain(allTerms, searcher);
+      idf = idfExp.getIdf();
     }
 
     @Override
@@ -167,10 +172,10 @@ public class MultiPhraseQuery extends Qu
     }
 
     @Override
-    public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
+    public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
       if (termArrays.size() == 0)                  // optimize zero-term case
         return null;
-
+      final IndexReader reader = context.reader;
       final Bits delDocs = reader.getDeletedDocs();
       
       PhraseQuery.PostingsAndFreq[] postingsFreqs = new PhraseQuery.PostingsAndFreq[termArrays.size()];
@@ -219,7 +224,7 @@ public class MultiPhraseQuery extends Qu
 
       if (slop == 0) {
         ExactPhraseScorer s = new ExactPhraseScorer(this, postingsFreqs, similarity,
-                                                    reader.norms(field));
+            reader.norms(field));
         if (s.noDocs) {
           return null;
         } else {
@@ -232,12 +237,12 @@ public class MultiPhraseQuery extends Qu
     }
 
     @Override
-    public Explanation explain(IndexReader reader, int doc)
+    public Explanation explain(AtomicReaderContext context, int doc)
       throws IOException {
       ComplexExplanation result = new ComplexExplanation();
       result.setDescription("weight("+getQuery()+" in "+doc+"), product of:");
 
-      Explanation idfExpl = new Explanation(idf, "idf("+getQuery()+")");
+      Explanation idfExpl = new Explanation(idf, "idf(" + field + ":" + idfExp.explain() +")");
 
       // explain query weight
       Explanation queryExpl = new Explanation();
@@ -263,7 +268,7 @@ public class MultiPhraseQuery extends Qu
       fieldExpl.setDescription("fieldWeight("+getQuery()+" in "+doc+
                                "), product of:");
 
-      Scorer scorer = scorer(reader, true, false);
+      Scorer scorer = scorer(context, ScorerContext.def());
       if (scorer == null) {
         return new Explanation(0.0f, "no matching docs");
       }
@@ -283,7 +288,7 @@ public class MultiPhraseQuery extends Qu
       fieldExpl.addDetail(idfExpl);
 
       Explanation fieldNormExpl = new Explanation();
-      byte[] fieldNorms = reader.norms(field);
+      byte[] fieldNorms = context.reader.norms(field);
       float fieldNorm =
         fieldNorms!=null ? similarity.decodeNormValue(fieldNorms[doc]) : 1.0f;
       fieldNormExpl.setValue(fieldNorm);

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java Wed Feb  9 09:35:27 2011
@@ -26,6 +26,7 @@ import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.queryParser.QueryParser;
 import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.util.PerReaderTermState;
 
 /**
  * An abstract {@link Query} that matches documents
@@ -33,7 +34,7 @@ import org.apache.lucene.util.AttributeS
  * FilteredTermsEnum} enumeration.
  *
  * <p>This query cannot be used directly; you must subclass
- * it and define {@link #getTermsEnum(IndexReader,AttributeSource)} to provide a {@link
+ * it and define {@link #getTermsEnum(Terms,AttributeSource)} to provide a {@link
  * FilteredTermsEnum} that iterates through the terms to be
  * matched.
  *
@@ -159,8 +160,8 @@ public abstract class MultiTermQuery ext
     }
     
     @Override
-    protected void addClause(BooleanQuery topLevel, Term term, int docCount, float boost) {
-      final TermQuery tq = new TermQuery(term, docCount);
+    protected void addClause(BooleanQuery topLevel, Term term, int docCount, float boost, PerReaderTermState states) {
+      final TermQuery tq = new TermQuery(term, states);
       tq.setBoost(boost);
       topLevel.add(tq, BooleanClause.Occur.SHOULD);
     }
@@ -200,8 +201,8 @@ public abstract class MultiTermQuery ext
     }
     
     @Override
-    protected void addClause(BooleanQuery topLevel, Term term, int docFreq, float boost) {
-      final Query q = new ConstantScoreQuery(new TermQuery(term, docFreq));
+    protected void addClause(BooleanQuery topLevel, Term term, int docFreq, float boost, PerReaderTermState states) {
+      final Query q = new ConstantScoreQuery(new TermQuery(term, states));
       q.setBoost(boost);
       topLevel.add(q, BooleanClause.Occur.SHOULD);
     }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java Wed Feb  9 09:35:27 2011
@@ -19,8 +19,9 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.index.DocsEnum;
@@ -104,7 +105,8 @@ public class MultiTermQueryWrapperFilter
    * results.
    */
   @Override
-  public DocIdSet getDocIdSet(IndexReader reader) throws IOException {
+  public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
+    final IndexReader reader = context.reader;
     final Fields fields = reader.fields();
     if (fields == null) {
       // reader has no fields
@@ -121,7 +123,7 @@ public class MultiTermQueryWrapperFilter
     assert termsEnum != null;
     if (termsEnum.next() != null) {
       // fill into a OpenBitSet
-      final OpenBitSet bitSet = new OpenBitSet(reader.maxDoc());
+      final OpenBitSet bitSet = new OpenBitSet(context.reader.maxDoc());
       int termCount = 0;
       final Bits delDocs = reader.getDeletedDocs();
       DocsEnum docsEnum = null;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/NumericRangeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/NumericRangeFilter.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/NumericRangeFilter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/NumericRangeFilter.java Wed Feb  9 09:35:27 2011
@@ -39,8 +39,6 @@ import org.apache.lucene.util.NumericUti
  * See {@link NumericRangeQuery} for details on how Lucene
  * indexes and searches numeric valued fields.
  *
- * @lucene.experimental
- *
  * @since 2.9
  **/
 public final class NumericRangeFilter<T extends Number> extends MultiTermQueryWrapperFilter<NumericRangeQuery<T>> {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/NumericRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/NumericRangeQuery.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/NumericRangeQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/NumericRangeQuery.java Wed Feb  9 09:35:27 2011
@@ -76,8 +76,6 @@ import org.apache.lucene.index.TermsEnum
  * BooleanQuery rewrite methods without changing
  * BooleanQuery's default max clause count.
  *
- * @lucene.experimental
- *
  * <br><h3>How it works</h3>
  *
  * <p>See the publication about <a target="_blank" href="http://www.panfmp.org">panFMP</a>,

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/PhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/PhraseQuery.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/PhraseQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/PhraseQuery.java Wed Feb  9 09:35:27 2011
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.Set;
 import java.util.ArrayList;
 
+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;
@@ -145,7 +146,7 @@ public class PhraseQuery extends Query {
 
     public PhraseWeight(IndexSearcher searcher)
       throws IOException {
-      this.similarity = getSimilarity(searcher);
+      this.similarity = searcher.getSimilarityProvider().get(field);
 
       idfExp = similarity.idfExplain(terms, searcher);
       idf = idfExp.getIdf();
@@ -174,10 +175,10 @@ public class PhraseQuery extends Query {
     }
 
     @Override
-    public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
+    public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
       if (terms.size() == 0)			  // optimize zero-term case
         return null;
-
+      final IndexReader reader = context.reader;
       PostingsAndFreq[] postingsFreqs = new PostingsAndFreq[terms.size()];
       final Bits delDocs = reader.getDeletedDocs();
       for (int i = 0; i < terms.size(); i++) {
@@ -206,7 +207,7 @@ public class PhraseQuery extends Query {
 
       if (slop == 0) {				  // optimize exact case
         ExactPhraseScorer s = new ExactPhraseScorer(this, postingsFreqs, similarity,
-                                                    reader.norms(field));
+            reader.norms(field));
         if (s.noDocs) {
           return null;
         } else {
@@ -215,12 +216,12 @@ public class PhraseQuery extends Query {
       } else {
         return
           new SloppyPhraseScorer(this, postingsFreqs, similarity, slop,
-                                 reader.norms(field));
+              reader.norms(field));
       }
     }
 
     @Override
-    public Explanation explain(IndexReader reader, int doc)
+    public Explanation explain(AtomicReaderContext context, int doc)
       throws IOException {
 
       Explanation result = new Explanation();
@@ -267,7 +268,7 @@ public class PhraseQuery extends Query {
       fieldExpl.setDescription("fieldWeight("+field+":"+query+" in "+doc+
                                "), product of:");
 
-      Scorer scorer = scorer(reader, true, false);
+      Scorer scorer = scorer(context, ScorerContext.def());
       if (scorer == null) {
         return new Explanation(0.0f, "no matching docs");
       }
@@ -287,7 +288,7 @@ public class PhraseQuery extends Query {
       fieldExpl.addDetail(idfExpl);
 
       Explanation fieldNormExpl = new Explanation();
-      byte[] fieldNorms = reader.norms(field);
+      byte[] fieldNorms = context.reader.norms(field);
       float fieldNorm =
         fieldNorms!=null ? similarity.decodeNormValue(fieldNorms[doc]) : 1.0f;
       fieldNormExpl.setValue(fieldNorm);

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/PhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/PhraseScorer.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/PhraseScorer.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/PhraseScorer.java Wed Feb  9 09:35:27 2011
@@ -40,9 +40,12 @@ abstract class PhraseScorer extends Scor
 
   private float freq; //phrase frequency in current doc as computed by phraseFreq().
 
+  protected final Similarity similarity;
+
   PhraseScorer(Weight weight, PhraseQuery.PostingsAndFreq[] postings,
       Similarity similarity, byte[] norms) {
-    super(similarity, weight);
+    super(weight);
+    this.similarity = similarity;
     this.norms = norms;
     this.value = weight.getValue();
 
@@ -105,8 +108,8 @@ abstract class PhraseScorer extends Scor
   @Override
   public float score() throws IOException {
     //System.out.println("scoring " + first.doc);
-    float raw = getSimilarity().tf(freq) * value; // raw score
-    return norms == null ? raw : raw * getSimilarity().decodeNormValue(norms[first.doc]); // normalize
+    float raw = similarity.tf(freq) * value; // raw score
+    return norms == null ? raw : raw * similarity.decodeNormValue(norms[first.doc]); // normalize
   }
 
   @Override

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/PositiveScoresOnlyCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/PositiveScoresOnlyCollector.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/PositiveScoresOnlyCollector.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/PositiveScoresOnlyCollector.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 which wraps another
@@ -43,8 +43,8 @@ public class PositiveScoresOnlyCollector
   }
 
   @Override
-  public void setNextReader(IndexReader reader, int docBase) throws IOException {
-    c.setNextReader(reader, docBase);
+  public void setNextReader(AtomicReaderContext context) throws IOException {
+    c.setNextReader(context);
   }
 
   @Override

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Query.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Query.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Query.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Query.java Wed Feb  9 09:35:27 2011
@@ -98,7 +98,7 @@ public abstract class Query implements j
     Query query = searcher.rewrite(this);
     Weight weight = query.createWeight(searcher);
     float sum = weight.sumOfSquaredWeights();
-    float norm = getSimilarity(searcher).queryNorm(sum);
+    float norm = searcher.getSimilarityProvider().queryNorm(sum);
     if (Float.isInfinite(norm) || Float.isNaN(norm))
       norm = 1.0f;
     weight.normalize(norm);
@@ -124,15 +124,6 @@ public abstract class Query implements j
     // needs to be implemented by query subclasses
     throw new UnsupportedOperationException();
   }
-  
-
-  /** Expert: Returns the Similarity implementation to be used for this query.
-   * Subclasses may override this method to specify their own Similarity
-   * implementation, perhaps one that delegates through that of the Searcher.
-   * By default the Searcher's Similarity implementation is returned.*/
-  public Similarity getSimilarity(IndexSearcher searcher) {
-    return searcher.getSimilarity();
-  }
 
   /** Returns a clone of this query. */
   @Override

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/QueryWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/QueryWrapperFilter.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/QueryWrapperFilter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/QueryWrapperFilter.java Wed Feb  9 09:35:27 2011
@@ -19,8 +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.search.Weight.ScorerContext;
 
 /** 
  * Constrains search results to only match those which also match a provided
@@ -48,12 +48,15 @@ public class QueryWrapperFilter extends 
   }
 
   @Override
-  public DocIdSet getDocIdSet(final IndexReader reader) throws IOException {
-    final Weight weight = query.weight(new IndexSearcher(reader));
+  public DocIdSet getDocIdSet(final AtomicReaderContext context) throws IOException {
+    // get a private context that is used to rewrite, createWeight and score eventually
+    assert context.reader.getTopReaderContext().isAtomic;
+    final AtomicReaderContext privateContext = (AtomicReaderContext) context.reader.getTopReaderContext();
+    final Weight weight = query.weight(new IndexSearcher(privateContext));
     return new DocIdSet() {
       @Override
       public DocIdSetIterator iterator() throws IOException {
-        return weight.scorer(reader, true, false);
+        return weight.scorer(privateContext, ScorerContext.def());
       }
       @Override
       public boolean isCacheable() { return false; }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ReqExclScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ReqExclScorer.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ReqExclScorer.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ReqExclScorer.java Wed Feb  9 09:35:27 2011
@@ -36,7 +36,7 @@ class ReqExclScorer extends Scorer {
    * @param exclDisi indicates exclusion.
    */
   public ReqExclScorer(Scorer reqScorer, DocIdSetIterator exclDisi) {
-    super(null); // No similarity used.
+    super(reqScorer.weight);
     this.reqScorer = reqScorer;
     this.exclDisi = exclDisi;
   }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ReqOptSumScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ReqOptSumScorer.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ReqOptSumScorer.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ReqOptSumScorer.java Wed Feb  9 09:35:27 2011
@@ -38,7 +38,7 @@ class ReqOptSumScorer extends Scorer {
       Scorer reqScorer,
       Scorer optScorer)
   {
-    super(null); // No similarity used.
+    super(reqScorer.weight);
     this.reqScorer = reqScorer;
     this.optScorer = optScorer;
   }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java Wed Feb  9 09:35:27 2011
@@ -38,19 +38,14 @@ public class ScoreCachingWrappingScorer 
   
   /** Creates a new instance by wrapping the given scorer. */
   public ScoreCachingWrappingScorer(Scorer scorer) {
-    super(scorer.getSimilarity());
+    super(scorer.weight);
     this.scorer = scorer;
   }
 
   @Override
-  protected boolean score(Collector collector, int max, int firstDocID) throws IOException {
+  public boolean score(Collector collector, int max, int firstDocID) throws IOException {
     return scorer.score(collector, max, firstDocID);
   }
-
-  @Override
-  public Similarity getSimilarity() {
-    return scorer.getSimilarity();
-  }
   
   @Override
   public float score() throws IOException {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ScoreDoc.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ScoreDoc.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ScoreDoc.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ScoreDoc.java Wed Feb  9 09:35:27 2011
@@ -24,7 +24,7 @@ public class ScoreDoc implements java.io
   public float score;
 
   /** Expert: A hit document's number.
-   * @see Searcher#doc(int)
+   * @see IndexSearcher#doc(int)
    */
   public int doc;
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Scorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Scorer.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Scorer.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Scorer.java Wed Feb  9 09:35:27 2011
@@ -40,31 +40,16 @@ import org.apache.lucene.search.BooleanC
  * with these scores.
  */
 public abstract class Scorer extends DocIdSetIterator {
-  private final Similarity similarity;
   protected final Weight weight;
 
-  /** Constructs a Scorer.
-   * @param similarity The <code>Similarity</code> implementation used by this scorer.
-   */
-  protected Scorer(Similarity similarity) {
-    this(similarity, null);
-  }
-  
   /**
    * Constructs a Scorer
-   * @param similarity The <code>Similarity</code> implementation used by this scorer.
-   * @param weight The scorers <code>Weight</code>
+   * @param weight The scorers <code>Weight</code>.
    */
-  protected Scorer(Similarity similarity, Weight weight) {
-    this.similarity = similarity;
+  protected Scorer(Weight weight) {
     this.weight = weight;
   }
 
-  /** Returns the Similarity implementation used by this scorer. */
-  public Similarity getSimilarity() {
-    return this.similarity;
-  }
-
   /** Scores and collects all matching documents.
    * @param collector The collector to which all matching documents are passed.
    */
@@ -90,7 +75,7 @@ public abstract class Scorer extends Doc
    *          this method.
    * @return true if more matching documents may remain.
    */
-  protected boolean score(Collector collector, int max, int firstDocID) throws IOException {
+  public boolean score(Collector collector, int max, int firstDocID) throws IOException {
     collector.setScorer(this);
     int doc = firstDocID;
     while (doc < max) {
@@ -172,7 +157,7 @@ public abstract class Scorer extends Doc
    * <p>
    * Note: this method will throw {@link UnsupportedOperationException} if no
    * associated {@link Weight} instance is provided to
-   * {@link #Scorer(Similarity, Weight)}
+   * {@link #Scorer(Weight)}
    * </p>
    * 
    * @lucene.experimental

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ScoringRewrite.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ScoringRewrite.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ScoringRewrite.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ScoringRewrite.java Wed Feb  9 09:35:27 2011
@@ -20,6 +20,7 @@ package org.apache.lucene.search;
 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.search.MultiTermQuery.RewriteMethod;
 
@@ -27,6 +28,7 @@ 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;
 
@@ -53,8 +55,9 @@ public abstract class ScoringRewrite<Q e
     }
     
     @Override
-    protected void addClause(BooleanQuery topLevel, Term term, int docCount, float boost) {
-      final TermQuery tq = new TermQuery(term, docCount);
+    protected void addClause(BooleanQuery topLevel, Term term, int docCount,
+        float boost, PerReaderTermState states) {
+      final TermQuery tq = new TermQuery(term, states);
       tq.setBoost(boost);
       topLevel.add(tq, BooleanClause.Occur.SHOULD);
     }
@@ -114,13 +117,13 @@ public abstract class ScoringRewrite<Q e
     final int size = col.terms.size();
     if (size > 0) {
       final int sort[] = col.terms.sort(col.termsEnum.getComparator());
-      final int[] docFreq = col.array.docFreq;
       final float[] boost = col.array.boost;
+      final PerReaderTermState[] termStates = col.array.termState;
       for (int i = 0; i < size; i++) {
         final int pos = sort[i];
         final Term term = placeholderTerm.createTerm(col.terms.get(pos, new BytesRef()));
-        assert reader.docFreq(term) == docFreq[pos];
-        addClause(result, term, docFreq[pos], query.getBoost() * boost[pos]);
+        assert reader.docFreq(term) == termStates[pos].docFreq();
+        addClause(result, term, termStates[pos].docFreq(), query.getBoost() * boost[pos], termStates[pos]);
       }
     }
     query.incTotalNumberOfTerms(size);
@@ -143,15 +146,17 @@ public abstract class ScoringRewrite<Q e
     @Override
     public boolean collect(BytesRef bytes) throws IOException {
       final int e = terms.add(bytes);
+      final TermState state = termsEnum.termState();
+      assert state != null; 
       if (e < 0 ) {
         // duplicate term: update docFreq
         final int pos = (-e)-1;
-        array.docFreq[pos] += termsEnum.docFreq();
+        array.termState[pos].register(state, readerContext.ord, termsEnum.docFreq());
         assert array.boost[pos] == boostAtt.getBoost() : "boost should be equal in all segment TermsEnums";
       } else {
         // new entry: we populate the entry initially
-        array.docFreq[e] = termsEnum.docFreq();
         array.boost[e] = boostAtt.getBoost();
+        array.termState[e] = new PerReaderTermState(topReaderContext, state, readerContext.ord, termsEnum.docFreq());
         ScoringRewrite.this.checkMaxClauseCount(terms.size());
       }
       return true;
@@ -160,8 +165,8 @@ public abstract class ScoringRewrite<Q e
   
   /** Special implementation of BytesStartArray that keeps parallel arrays for boost and docFreq */
   static final class TermFreqBoostByteStart extends DirectBytesStartArray  {
-    int[] docFreq;
     float[] boost;
+    PerReaderTermState[] termState;
     
     public TermFreqBoostByteStart(int initSize) {
       super(initSize);
@@ -171,24 +176,28 @@ public abstract class ScoringRewrite<Q e
     public int[] init() {
       final int[] ord = super.init();
       boost = new float[ArrayUtil.oversize(ord.length, RamUsageEstimator.NUM_BYTES_FLOAT)];
-      docFreq = new int[ArrayUtil.oversize(ord.length, RamUsageEstimator.NUM_BYTES_INT)];
-      assert boost.length >= ord.length && docFreq.length >= ord.length;
+      termState = new PerReaderTermState[ArrayUtil.oversize(ord.length, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+      assert termState.length >= ord.length && boost.length >= ord.length;
       return ord;
     }
 
     @Override
     public int[] grow() {
       final int[] ord = super.grow();
-      docFreq = ArrayUtil.grow(docFreq, ord.length);
       boost = ArrayUtil.grow(boost, ord.length);
-      assert boost.length >= ord.length && docFreq.length >= ord.length;
+      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 && boost.length >= ord.length;
       return ord;
     }
 
     @Override
     public int[] clear() {
      boost = null;
-     docFreq = null;
+     termState = null;
      return super.clear();
     }