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

svn commit: r1058390 [5/16] - in /lucene/dev/branches/bulkpostings: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/ dev-tools/idea/.idea/ dev-tools/idea/.idea/libraries/ dev-tools/idea/lucene/ dev-tools/idea/lucene/contrib/ dev-tools/idea/lucene/contr...

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java Thu Jan 13 02:09:33 2011
@@ -18,14 +18,17 @@ 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 java.io.IOException;
 import java.util.Set;
 
 /**
- * A query that wraps a filter and simply returns a constant score equal to the
- * query boost for every document in the filter.
+ * A query that wraps another query or a filter and simply returns a constant score equal to the
+ * query boost for every document that matches the filter or query.
+ * For queries it therefore simply strips of all scores and returns a constant one.
  *
  * <p><b>NOTE</b>: if the wrapped filter is an instance of
  * {@link CachingWrapperFilter}, you'll likely want to
@@ -35,34 +38,72 @@ import java.util.Set;
  */
 public class ConstantScoreQuery extends Query {
   protected final Filter filter;
+  protected final Query query;
 
+  /** Strips off scores from the passed in Query. The hits will get a constant score
+   * dependent on the boost factor of this query. */
+  public ConstantScoreQuery(Query query) {
+    if (query == null)
+      throw new NullPointerException("Query may not be null");
+    this.filter = null;
+    this.query = query;
+  }
+
+  /** Wraps a Filter as a Query. The hits will get a constant score
+   * dependent on the boost factor of this query.
+   * If you simply want to strip off scores from a Query, no longer use
+   * {@code new ConstantScoreQuery(new QueryWrapperFilter(query))}, instead
+   * use {@link #ConstantScoreQuery(Query)}!
+   */
   public ConstantScoreQuery(Filter filter) {
-    this.filter=filter;
+    if (filter == null)
+      throw new NullPointerException("Filter may not be null");
+    this.filter = filter;
+    this.query = null;
   }
 
-  /** Returns the encapsulated filter */
+  /** Returns the encapsulated filter, returns {@code null} if a query is wrapped. */
   public Filter getFilter() {
     return filter;
   }
 
+  /** Returns the encapsulated query, returns {@code null} if a filter is wrapped. */
+  public Query getQuery() {
+    return query;
+  }
+
   @Override
   public Query rewrite(IndexReader reader) throws IOException {
+    if (query != null) {
+      Query rewritten = query.rewrite(reader);
+      if (rewritten != query) {
+        rewritten = new ConstantScoreQuery(rewritten);
+        rewritten.setBoost(this.getBoost());
+        return rewritten;
+      }
+    }
     return this;
   }
 
   @Override
   public void extractTerms(Set<Term> terms) {
-    // OK to not add any terms when used for MultiSearcher,
-    // but may not be OK for highlighting
+    // TODO: OK to not add any terms when wrapped a filter
+    // and used with MultiSearcher, but may not be OK for
+    // highlighting.
+    // If a query was wrapped, we delegate to query.
+    if (query != null)
+      query.extractTerms(terms);
   }
 
   protected class ConstantWeight extends Weight {
-    private Similarity similarity;
+    private final Weight innerWeight;
+    private final Similarity similarity;
     private float queryNorm;
     private float queryWeight;
     
-    public ConstantWeight(Searcher searcher) {
+    public ConstantWeight(IndexSearcher searcher) throws IOException {
       this.similarity = getSimilarity(searcher);
+      this.innerWeight = (query == null) ? null : query.createWeight(searcher);
     }
 
     @Override
@@ -77,6 +118,8 @@ public class ConstantScoreQuery extends 
 
     @Override
     public float sumOfSquaredWeights() throws IOException {
+      // we calculate sumOfSquaredWeights of the inner weight, but ignore it (just to initialize everything)
+      if (innerWeight != null) innerWeight.sumOfSquaredWeights();
       queryWeight = getBoost();
       return queryWeight * queryWeight;
     }
@@ -85,31 +128,48 @@ public class ConstantScoreQuery extends 
     public void normalize(float norm) {
       this.queryNorm = norm;
       queryWeight *= this.queryNorm;
+      // we normalize the inner weight, but ignore it (just to initialize everything)
+      if (innerWeight != null) innerWeight.normalize(norm);
     }
 
     @Override
-    public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
-      return new ConstantScorer(similarity, reader, this);
+    public Scorer scorer(AtomicReaderContext context,  boolean scoreDocsInOrder, boolean topScorer) throws IOException {
+      final DocIdSetIterator disi;
+      if (filter != null) {
+        assert query == null;
+        final DocIdSet dis = filter.getDocIdSet(context);
+        if (dis == null)
+          return null;
+        disi = dis.iterator();
+      } else {
+        assert query != null && innerWeight != null;
+        disi =
+          innerWeight.scorer(context, scoreDocsInOrder, topScorer);
+      }
+      if (disi == null)
+        return null;
+      return new ConstantScorer(similarity, disi, this);
+    }
+    
+    @Override
+    public boolean scoresDocsOutOfOrder() {
+      return (innerWeight != null) ? innerWeight.scoresDocsOutOfOrder() : false;
     }
 
     @Override
-    public Explanation explain(IndexReader reader, int doc) throws IOException {
-      
-      ConstantScorer cs = new ConstantScorer(similarity, reader, this);
-      boolean exists = cs.docIdSetIterator.advance(doc) == doc;
-
-      ComplexExplanation result = new ComplexExplanation();
+    public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
+      final Scorer cs = scorer(context, true, false);
+      final boolean exists = (cs != null && cs.advance(doc) == doc);
 
+      final ComplexExplanation result = new ComplexExplanation();
       if (exists) {
-        result.setDescription("ConstantScoreQuery(" + filter
-        + "), product of:");
+        result.setDescription(ConstantScoreQuery.this.toString() + ", product of:");
         result.setValue(queryWeight);
         result.setMatch(Boolean.TRUE);
         result.addDetail(new Explanation(getBoost(), "boost"));
-        result.addDetail(new Explanation(queryNorm,"queryNorm"));
+        result.addDetail(new Explanation(queryNorm, "queryNorm"));
       } else {
-        result.setDescription("ConstantScoreQuery(" + filter
-        + ") doesn't match id " + doc);
+        result.setDescription(ConstantScoreQuery.this.toString() + " doesn't match id " + doc);
         result.setValue(0);
         result.setMatch(Boolean.FALSE);
       }
@@ -120,22 +180,11 @@ public class ConstantScoreQuery extends 
   protected class ConstantScorer extends Scorer {
     final DocIdSetIterator docIdSetIterator;
     final float theScore;
-    int doc = -1;
 
-    public ConstantScorer(Similarity similarity, IndexReader reader, Weight w) throws IOException {
+    public ConstantScorer(Similarity similarity, DocIdSetIterator docIdSetIterator, Weight w) throws IOException {
       super(similarity,w);
       theScore = w.getValue();
-      DocIdSet docIdSet = filter.getDocIdSet(reader);
-      if (docIdSet == null) {
-        docIdSetIterator = DocIdSet.EMPTY_DOCIDSET.iterator();
-      } else {
-        DocIdSetIterator iter = docIdSet.iterator();
-        if (iter == null) {
-          docIdSetIterator = DocIdSet.EMPTY_DOCIDSET.iterator();
-        } else {
-          docIdSetIterator = iter;
-        }
-      }
+      this.docIdSetIterator = docIdSetIterator;
     }
 
     @Override
@@ -162,34 +211,86 @@ public class ConstantScoreQuery extends 
     public int advance(int target) throws IOException {
       return docIdSetIterator.advance(target);
     }
+    
+    private Collector wrapCollector(final Collector collector) {
+      return new Collector() {
+        @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));
+        }
+        
+        @Override
+        public void collect(int doc) throws IOException {
+          collector.collect(doc);
+        }
+        
+        @Override
+        public void setNextReader(IndexReader reader, int docBase) throws IOException {
+          collector.setNextReader(reader, docBase);
+        }
+        
+        @Override
+        public boolean acceptsDocsOutOfOrder() {
+          return collector.acceptsDocsOutOfOrder();
+        }
+      };
+    }
+
+    // this optimization allows out of order scoring as top scorer!
+    @Override
+    public void score(Collector collector) throws IOException {
+      if (docIdSetIterator instanceof Scorer) {
+        ((Scorer) docIdSetIterator).score(wrapCollector(collector));
+      } else {
+        super.score(collector);
+      }
+    }
+
+    // this optimization allows out of order scoring as top scorer,
+    @Override
+    public boolean score(Collector collector, int max, int firstDocID) throws IOException {
+      if (docIdSetIterator instanceof Scorer) {
+        return ((Scorer) docIdSetIterator).score(wrapCollector(collector), max, firstDocID);
+      } else {
+        return super.score(collector, max, firstDocID);
+      }
+    }
   }
 
   @Override
-  public Weight createWeight(Searcher searcher) {
+  public Weight createWeight(IndexSearcher searcher) throws IOException {
     return new ConstantScoreQuery.ConstantWeight(searcher);
   }
 
-  /** Prints a user-readable version of this query. */
   @Override
   public String toString(String field) {
-    return "ConstantScore(" + filter.toString() + ")"
-      + (getBoost()==1.0 ? "" : "^" + getBoost());
+    return new StringBuilder("ConstantScore(")
+      .append((query == null) ? filter.toString() : query.toString(field))
+      .append(')')
+      .append(ToStringUtils.boost(getBoost()))
+      .toString();
   }
 
-  /** Returns true if <code>o</code> is equal to this. */
   @Override
   public boolean equals(Object o) {
     if (this == o) return true;
-    if (!(o instanceof ConstantScoreQuery)) return false;
-    ConstantScoreQuery other = (ConstantScoreQuery)o;
-    return this.getBoost()==other.getBoost() && filter.equals(other.filter);
+    if (!super.equals(o))
+      return false;
+    if (o instanceof ConstantScoreQuery) {
+      final ConstantScoreQuery other = (ConstantScoreQuery) o;
+      return 
+        ((this.filter == null) ? other.filter == null : this.filter.equals(other.filter)) &&
+        ((this.query == null) ? other.query == null : this.query.equals(other.query));
+    }
+    return false;
   }
 
-  /** Returns a hash code value for this object. */
   @Override
   public int hashCode() {
-    // Simple add is OK since no existing filter hashcode has a float component.
-    return filter.hashCode() + Float.floatToIntBits(getBoost());
+    return 31 * super.hashCode() +
+      ((query == null) ? filter : query).hashCode();
   }
 
 }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/DefaultSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/DefaultSimilarity.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/DefaultSimilarity.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/DefaultSimilarity.java Thu Jan 13 02:09:33 2011
@@ -37,13 +37,7 @@ 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>. */

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java Thu Jan 13 02:09:33 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;
 
 /**
@@ -101,7 +102,7 @@ public class DisjunctionMaxQuery extends
     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. */
-    public DisjunctionMaxWeight(Searcher searcher) throws IOException {
+    public DisjunctionMaxWeight(IndexSearcher searcher) throws IOException {
       this.similarity = searcher.getSimilarity();
       for (Query disjunctQuery : disjuncts) {
         weights.add(disjunctQuery.createWeight(searcher));
@@ -141,12 +142,12 @@ public class DisjunctionMaxQuery extends
 
     /* Create the scorer used to score our associated DisjunctionMaxQuery */
     @Override
-    public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder,
+    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
         boolean topScorer) 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, true, false);
         if (subScorer != null && subScorer.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
           scorers[idx++] = subScorer;
         }
@@ -158,13 +159,13 @@ public class DisjunctionMaxQuery extends
 
     /* 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);
@@ -180,7 +181,7 @@ public class DisjunctionMaxQuery extends
 
   /* Create the Weight used to score us */
   @Override
-  public Weight createWeight(Searcher searcher) throws IOException {
+  public Weight createWeight(IndexSearcher searcher) throws IOException {
     return new DisjunctionMaxWeight(searcher);
   }
 

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/DisjunctionSumScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/DisjunctionSumScorer.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/DisjunctionSumScorer.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/DisjunctionSumScorer.java Thu Jan 13 02:09:33 2011
@@ -123,7 +123,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/bulkpostings/lucene/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java Thu Jan 13 02:09:33 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;
@@ -532,7 +533,7 @@ public abstract class FieldCacheRangeFil
     @Override
     public DocIdSetIterator iterator() throws IOException {
 
-      final Bits skipDocs = canIgnoreDeletedDocs ? null : MultiFields.getDeletedDocs(reader);
+      final Bits skipDocs = canIgnoreDeletedDocs ? null : reader.getDeletedDocs();
 
       if (skipDocs == null) {
         // Specialization optimization disregard deletions

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java Thu Jan 13 02:09:33 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/bulkpostings/lucene/src/java/org/apache/lucene/search/FieldDoc.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/FieldDoc.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/FieldDoc.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/FieldDoc.java Thu Jan 13 02:09:33 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;
 

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/FieldValueHitQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/FieldValueHitQueue.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/FieldValueHitQueue.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/FieldValueHitQueue.java Thu Jan 13 02:09:33 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> {

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Filter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Filter.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Filter.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Filter.java Thu Jan 13 02:09:33 2011
@@ -19,7 +19,7 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.util.DocIdBitSet;
 
 /** 
@@ -38,10 +38,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. Note, it is likely that the provided reader does not
+   * @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 +52,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/bulkpostings/lucene/src/java/org/apache/lucene/search/FilteredQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/FilteredQuery.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/FilteredQuery.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/FilteredQuery.java Thu Jan 13 02:09:33 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;
 
@@ -59,7 +60,7 @@ extends Query {
    * This is accomplished by overriding the Scorer returned by the Weight.
    */
   @Override
-  public Weight createWeight(final Searcher searcher) throws IOException {
+  public Weight createWeight(final IndexSearcher searcher) throws IOException {
     final Weight weight = query.createWeight (searcher);
     final Similarity similarity = query.getSimilarity(searcher);
     return new Weight() {
@@ -81,7 +82,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,7 +112,7 @@ extends Query {
 
       // return a filtering scorer
       @Override
-      public Scorer scorer(IndexReader indexReader, boolean scoreDocsInOrder, boolean topScorer)
+      public Scorer scorer(AtomicReaderContext indexReader, boolean scoreDocsInOrder, boolean topScorer)
           throws IOException {
         final Scorer scorer = weight.scorer(indexReader, true, false);
         if (scorer == null) {

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java Thu Jan 13 02:09:33 2011
@@ -321,7 +321,7 @@ public final class FuzzyTermsEnum extend
       
       // TODO: benchmark doing this backwards
       for (int i = 1; i < matchers.length; i++)
-        if (matchers[i].run(term.bytes, 0, term.length)) {
+        if (matchers[i].run(term.bytes, term.offset, term.length)) {
           // this sucks, we convert just to score based on length.
           if (codePointCount == -1) {
             codePointCount = UnicodeUtil.codePointCount(term);

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/IndexSearcher.java Thu Jan 13 02:09:33 2011
@@ -18,16 +18,28 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.lucene.document.Document;
 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.store.Directory;
+import org.apache.lucene.store.NIOFSDirectory;    // javadoc
 import org.apache.lucene.util.ReaderUtil;
+import org.apache.lucene.util.ThreadInterruptedException;
 
 /** Implements search over a single IndexReader.
  *
@@ -44,14 +56,21 @@ import org.apache.lucene.util.ReaderUtil
  * synchronize on the <code>IndexSearcher</code> instance;
  * use your own (non-Lucene) objects instead.</p>
  */
-public class IndexSearcher extends Searcher {
-  IndexReader reader;
+public class IndexSearcher {
+  final IndexReader reader; // package private for testing!
   private boolean closeReader;
   
   // NOTE: these members might change in incompatible ways
   // in the next release
-  protected IndexReader[] subReaders;
-  protected int[] docStarts;
+  protected final ReaderContext readerContext;
+  protected final AtomicReaderContext[] leafContexts;
+
+  // These are only used for multi-threaded search
+  private final ExecutorService executor;
+  protected final IndexSearcher[] subSearchers;
+
+  /** The Similarity implementation used by this searcher. */
+  private Similarity similarity = Similarity.getDefault();
 
   /** Creates a searcher searching the index in the named
    *  directory, with readOnly=true
@@ -60,7 +79,7 @@ public class IndexSearcher extends Searc
    * @throws IOException if there is a low-level IO error
    */
   public IndexSearcher(Directory path) throws CorruptIndexException, IOException {
-    this(IndexReader.open(path, true), true);
+    this(IndexReader.open(path, true), true, null);
   }
 
   /** Creates a searcher searching the index in the named
@@ -75,102 +94,313 @@ public class IndexSearcher extends Searc
    * @throws IOException if there is a low-level IO error
    */
   public IndexSearcher(Directory path, boolean readOnly) throws CorruptIndexException, IOException {
-    this(IndexReader.open(path, readOnly), true);
+    this(IndexReader.open(path, readOnly), true, null);
   }
 
   /** Creates a searcher searching the provided index. */
   public IndexSearcher(IndexReader r) {
-    this(r, false);
+    this(r, false, null);
   }
 
-  /** Expert: directly specify the reader, subReaders and
-   *  their docID starts.
+  /** 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>).
    * 
    * @lucene.experimental */
-  public IndexSearcher(IndexReader reader, IndexReader[] subReaders, int[] docStarts) {
-    this.reader = reader;
-    this.subReaders = subReaders;
-    this.docStarts = docStarts;
-    closeReader = false;
+  public IndexSearcher(IndexReader r, ExecutorService executor) {
+    this(r, false, executor);
   }
-  
-  private IndexSearcher(IndexReader r, boolean closeReader) {
-    reader = r;
-    this.closeReader = closeReader;
 
-    List<IndexReader> subReadersList = new ArrayList<IndexReader>();
-    gatherSubReaders(subReadersList, reader);
-    subReaders = subReadersList.toArray(new IndexReader[subReadersList.size()]);
-    docStarts = new int[subReaders.length];
-    int maxDoc = 0;
-    for (int i = 0; i < subReaders.length; i++) {
-      docStarts[i] = maxDoc;
-      maxDoc += subReaders[i].maxDoc();
-    }
+  /**
+   * 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);
   }
 
-  protected void gatherSubReaders(List<IndexReader> allSubReaders, IndexReader r) {
-    ReaderUtil.gatherSubReaders(allSubReaders, r);
+  /**
+   * 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(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;
+    this.readerContext = context;
+    leafContexts = ReaderUtil.leaves(context);
+    
+    if (executor == null) {
+      subSearchers = null;
+    } else {
+      subSearchers = new IndexSearcher[this.leafContexts.length];
+      for (int i = 0; i < subSearchers.length; i++) {
+        if (leafContexts[i].reader == context.reader) {
+          subSearchers[i] = this;
+        } else {
+          subSearchers[i] = new IndexSearcher(context, leafContexts[i]);
+        }
+      }
+    }
+  }
+  
+  /* Ctor for concurrent sub-searchers searching only on a specific leaf of the given top-reader context
+   * - instead of searching over all leaves this searcher only searches a single leaf searcher slice. Hence, 
+   * for scorer and filter this looks like an ordinary search in the hierarchy such that there is no difference
+   * between single and multi-threaded */
+  private IndexSearcher(ReaderContext topLevel, AtomicReaderContext leaf) {
+    readerContext = topLevel;
+    reader = topLevel.reader;
+    leafContexts = new AtomicReaderContext[] {leaf};
+    executor = null;
+    subSearchers = null;
+    closeReader = false;
+  }
+  
   /** Return the {@link IndexReader} this searches. */
   public IndexReader getIndexReader() {
     return reader;
   }
 
+  /** Expert: Returns one greater than the largest possible document number.
+   * 
+   * @see org.apache.lucene.index.IndexReader#maxDoc()
+   */
+  public int maxDoc() {
+    return reader.maxDoc();
+  }
+
+  /** Returns total docFreq for this term. */
+  public int docFreq(final Term term) throws IOException {
+    if (executor == null) {
+      return reader.docFreq(term);
+    } else {
+      final ExecutionHelper<Integer> runner = new ExecutionHelper<Integer>(executor);
+      for(int i = 0; i < subSearchers.length; i++) {
+        final IndexSearcher searchable = subSearchers[i];
+        runner.submit(new Callable<Integer>() {
+            public Integer call() throws IOException {
+              return Integer.valueOf(searchable.docFreq(term));
+            }
+          });
+      }
+      int docFreq = 0;
+      for (Integer num : runner) {
+        docFreq += num.intValue();
+      }
+      return docFreq;
+    }
+  }
+
+  /* Sugar for .getIndexReader().document(docID) */
+  public Document doc(int docID) throws CorruptIndexException, IOException {
+    return reader.document(docID);
+  }
+  
+  /* Sugar for .getIndexReader().document(docID, fieldSelector) */
+  public Document doc(int docID, FieldSelector fieldSelector) throws CorruptIndexException, IOException {
+    return reader.document(docID, fieldSelector);
+  }
+  
+  /** Expert: Set the Similarity implementation used by this Searcher.
+   *
+   * @see Similarity#setDefault(Similarity)
+   */
+  public void setSimilarity(Similarity similarity) {
+    this.similarity = similarity;
+  }
+
+  public Similarity getSimilarity() {
+    return similarity;
+  }
+
   /**
    * Note that the underlying IndexReader is not closed, if
    * IndexSearcher was constructed with IndexSearcher(IndexReader r).
    * If the IndexReader was supplied implicitly by specifying a directory, then
-   * the IndexReader gets closed.
+   * the IndexReader is closed.
    */
-  @Override
   public void close() throws IOException {
-    if(closeReader)
+    if (closeReader) {
       reader.close();
+    }
   }
 
-  // inherit javadoc
-  @Override
-  public int docFreq(Term term) throws IOException {
-    return reader.docFreq(term);
+  /** Finds the top <code>n</code>
+   * hits for <code>query</code>.
+   *
+   * @throws BooleanQuery.TooManyClauses
+   */
+  public TopDocs search(Query query, int n)
+    throws IOException {
+    return search(query, null, n);
   }
 
-  // inherit javadoc
-  @Override
-  public Document doc(int i) throws CorruptIndexException, IOException {
-    return reader.document(i);
+
+  /** Finds the top <code>n</code>
+   * hits for <code>query</code>, applying <code>filter</code> if non-null.
+   *
+   * @throws BooleanQuery.TooManyClauses
+   */
+  public TopDocs search(Query query, Filter filter, int n)
+    throws IOException {
+    return search(createWeight(query), filter, n);
   }
-  
-  // inherit javadoc
-  @Override
-  public Document doc(int i, FieldSelector fieldSelector) throws CorruptIndexException, IOException {
-	    return reader.document(i, fieldSelector);
+
+  /** Lower-level search API.
+   *
+   * <p>{@link Collector#collect(int)} is called for every matching
+   * document.
+   * <br>Collector-based access to remote indexes is discouraged.
+   *
+   * <p>Applications should only use this if they need <i>all</i> of the
+   * matching documents.  The high-level search API ({@link
+   * IndexSearcher#search(Query, Filter, int)}) is usually more efficient, as it skips
+   * non-high-scoring hits.
+   *
+   * @param query to match documents
+   * @param filter if non-null, used to permit documents to be collected.
+   * @param results to receive hits
+   * @throws BooleanQuery.TooManyClauses
+   */
+  public void search(Query query, Filter filter, Collector results)
+    throws IOException {
+    search(createWeight(query), filter, results);
+  }
+
+  /** Lower-level search API.
+  *
+  * <p>{@link Collector#collect(int)} is called for every matching document.
+  *
+  * <p>Applications should only use this if they need <i>all</i> of the
+  * matching documents.  The high-level search API ({@link
+  * 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
+  * between 0 and 1.
+  * @throws BooleanQuery.TooManyClauses
+  */
+  public void search(Query query, Collector results)
+    throws IOException {
+    search(createWeight(query), null, results);
   }
   
-  // inherit javadoc
-  @Override
-  public int maxDoc() throws IOException {
-    return reader.maxDoc();
+  /** 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>NOTE: this does not compute scores by default; use
+   * {@link IndexSearcher#setDefaultFieldSortScoring} to
+   * enable scoring.
+   *
+   * @throws BooleanQuery.TooManyClauses
+   */
+  public TopFieldDocs search(Query query, Filter filter, int n,
+                             Sort sort) throws IOException {
+    return search(createWeight(query), filter, n, sort);
   }
 
-  // inherit javadoc
-  @Override
-  public TopDocs search(Weight weight, Filter filter, int nDocs) throws IOException {
+  /**
+   * Search implementation with arbitrary sorting and no filter.
+   * @param query The query to search for
+   * @param n Return only the top n results
+   * @param sort The {@link org.apache.lucene.search.Sort} object
+   * @return The top docs, sorted according to the supplied {@link org.apache.lucene.search.Sort} instance
+   * @throws IOException
+   */
+  public TopFieldDocs search(Query query, int n,
+                             Sort sort) throws IOException {
+    return search(createWeight(query), null, n, sort);
+  }
 
-    int limit = reader.maxDoc();
-    if (limit == 0) {
-      limit = 1;
-    }
-    nDocs = Math.min(nDocs, limit);
+  /** 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 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();
+    } 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 < subSearchers.length; i++) { // search each sub
+        runner.submit(
+                      new SearcherCallableNoSort(lock, subSearchers[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());
+      }
+
+      final ScoreDoc[] scoreDocs = new ScoreDoc[hq.size()];
+      for (int i = hq.size() - 1; i >= 0; i--) // put docs in array
+        scoreDocs[i] = hq.pop();
 
-    TopScoreDocCollector collector = TopScoreDocCollector.create(nDocs, !weight.scoresDocsOutOfOrder());
-    search(weight, filter, collector);
-    return collector.topDocs();
+      return new TopDocs(totalHits, scoreDocs, maxScore);
+    }
   }
 
-  @Override
-  public TopFieldDocs search(Weight weight, Filter filter,
+  /** 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
+   * IndexSearcher#search(Query,Filter,int,Sort)} instead.
+   * 
+   * @throws BooleanQuery.TooManyClauses
+   */
+  protected TopFieldDocs search(Weight weight, Filter filter,
       final int nDocs, Sort sort) throws IOException {
     return search(weight, filter, nDocs, sort, true);
   }
@@ -186,48 +416,96 @@ public class IndexSearcher extends Searc
    * then pass that to {@link #search(Weight, Filter,
    * Collector)}.</p>
    */
-  public TopFieldDocs search(Weight weight, Filter filter, int nDocs,
+  protected TopFieldDocs search(Weight weight, Filter filter, int nDocs,
                              Sort sort, boolean fillFields)
       throws IOException {
 
-    int limit = reader.maxDoc();
-    if (limit == 0) {
-      limit = 1;
-    }
-    nDocs = Math.min(nDocs, limit);
+    if (sort == null) throw new NullPointerException();
 
-    TopFieldCollector collector = TopFieldCollector.create(sort, nDocs,
-        fillFields, fieldSortDoTrackScores, fieldSortDoMaxScore, !weight.scoresDocsOutOfOrder());
-    search(weight, filter, collector);
-    return (TopFieldDocs) collector.topDocs();
+    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();
+    } 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 < subSearchers.length; i++) { // search each sub
+        runner.submit(
+                      new SearcherCallableWithSort(lock, subSearchers[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());
+      }
+      final ScoreDoc[] scoreDocs = new ScoreDoc[hq.size()];
+      for (int i = hq.size() - 1; i >= 0; i--) // put docs in array
+        scoreDocs[i] = hq.pop();
+
+      return new TopFieldDocs(totalHits, scoreDocs, hq.getFields(), maxScore);
+    }
   }
 
-  @Override
-  public void search(Weight weight, Filter filter, Collector collector)
+  /**
+   * Lower-level search API.
+   * 
+   * <p>
+   * {@link Collector#collect(int)} is called for every document. <br>
+   * Collector-based access to remote indexes is discouraged.
+   * 
+   * <p>
+   * Applications should only use this if they need <i>all</i> of the matching
+   * documents. The high-level search API ({@link IndexSearcher#search(Query,int)}) is
+   * usually more efficient, as it skips non-high-scoring hits.
+   * 
+   * @param weight
+   *          to match documents
+   * @param filter
+   *          if non-null, used to permit documents to be collected.
+   * @param collector
+   *          to receive hits
+   * @throws BooleanQuery.TooManyClauses
+   */
+  protected void search(Weight weight, Filter filter, Collector collector)
       throws IOException {
-    
+
+    // TODO: should we make this
+    // threaded...?  the Collector could be sync'd?
+
+    // 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 < leafContexts.length; i++) { // search each subreader
+        collector.setNextReader(leafContexts[i].reader, leafContexts[i].docBase);
+        Scorer scorer = weight.scorer(leafContexts[i], !collector.acceptsDocsOutOfOrder(), true);
         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 < leafContexts.length; i++) { // search each subreader
+        collector.setNextReader(leafContexts[i].reader, leafContexts[i].docBase);
+        searchWithFilter(leafContexts[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, true, false);
     if (scorer == null) {
       return;
     }
@@ -236,7 +514,7 @@ public class IndexSearcher extends Searc
     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;
@@ -268,7 +546,9 @@ public class IndexSearcher extends Searc
     }
   }
 
-  @Override
+  /** Expert: called to re-write queries into primitive queries.
+   * @throws BooleanQuery.TooManyClauses
+   */
   public Query rewrite(Query original) throws IOException {
     Query query = original;
     for (Query rewrittenQuery = query.rewrite(reader); rewrittenQuery != query;
@@ -278,12 +558,34 @@ public class IndexSearcher extends Searc
     return query;
   }
 
-  @Override
-  public Explanation explain(Weight weight, int doc) throws IOException {
-    int n = ReaderUtil.subIndex(doc, docStarts);
-    int deBasedDoc = doc - docStarts[n];
+  /** Returns an Explanation that describes how <code>doc</code> scored against
+   * <code>query</code>.
+   *
+   * <p>This is intended to be used in developing Similarity implementations,
+   * and, for good performance, should not be displayed with every hit.
+   * Computing an explanation is as expensive as executing the query over the
+   * entire index.
+   */
+  public Explanation explain(Query query, int doc) throws IOException {
+    return explain(createWeight(query), doc);
+  }
+
+  /** Expert: low-level implementation method
+   * Returns an Explanation that describes how <code>doc</code> scored against
+   * <code>weight</code>.
+   *
+   * <p>This is intended to be used in developing Similarity implementations,
+   * and, for good performance, should not be displayed with every hit.
+   * Computing an explanation is as expensive as executing the query over the
+   * entire index.
+   * <p>Applications should call {@link IndexSearcher#explain(Query, int)}.
+   * @throws BooleanQuery.TooManyClauses
+   */
+  protected Explanation explain(Weight weight, int doc) throws IOException {
+    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;
@@ -305,4 +607,159 @@ public class IndexSearcher extends Searc
     fieldSortDoTrackScores = doTrackScores;
     fieldSortDoMaxScore = doMaxScore;
   }
+
+  /**
+   * creates a weight for <code>query</code>
+   * @return new weight
+   */
+  protected Weight createWeight(Query query) throws IOException {
+    return query.weight(this);
+  }
+
+  /**
+   * Returns this searchers the top-level {@link ReaderContext}.
+   * @see IndexReader#getTopReaderContext()
+   */
+  /* Sugar for .getIndexReader().getTopReaderContext() */
+  public ReaderContext getTopReaderContext() {
+    return readerContext;
+  }
+
+  /**
+   * A thread subclass for searching a single searchable 
+   */
+  private static final class SearcherCallableNoSort implements Callable<TopDocs> {
+
+    private final Lock lock;
+    private final IndexSearcher searchable;
+    private final Weight weight;
+    private final Filter filter;
+    private final int nDocs;
+    private final HitQueue hq;
+
+    public SearcherCallableNoSort(Lock lock, IndexSearcher searchable, Weight weight,
+        Filter filter, int nDocs, HitQueue hq) {
+      this.lock = lock;
+      this.searchable = searchable;
+      this.weight = weight;
+      this.filter = filter;
+      this.nDocs = nDocs;
+      this.hq = hq;
+    }
+
+    public TopDocs call() throws IOException {
+      final TopDocs docs = searchable.search (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];
+        //it would be so nice if we had a thread-safe insert 
+        lock.lock();
+        try {
+          if (scoreDoc == hq.insertWithOverflow(scoreDoc))
+            break;
+        } finally {
+          lock.unlock();
+        }
+      }
+      return docs;
+    }
+  }
+
+
+  /**
+   * A thread subclass for searching a single searchable 
+   */
+  private static final class SearcherCallableWithSort implements Callable<TopFieldDocs> {
+
+    private final Lock lock;
+    private final IndexSearcher searchable;
+    private final Weight weight;
+    private final Filter filter;
+    private final int nDocs;
+    private final FieldDocSortedHitQueue hq;
+    private final Sort sort;
+
+    public SearcherCallableWithSort(Lock lock, IndexSearcher searchable, Weight weight,
+        Filter filter, int nDocs, FieldDocSortedHitQueue hq, Sort sort) {
+      this.lock = lock;
+      this.searchable = searchable;
+      this.weight = weight;
+      this.filter = filter;
+      this.nDocs = nDocs;
+      this.hq = hq;
+      this.sort = sort;
+    }
+
+    public TopFieldDocs call() throws IOException {
+      final TopFieldDocs docs = searchable.search (weight, filter, nDocs, sort);
+      lock.lock();
+      try {
+        hq.setFields(docs.fields);
+      } finally {
+        lock.unlock();
+      }
+
+      final ScoreDoc[] scoreDocs = docs.scoreDocs;
+      for (int j = 0; j < scoreDocs.length; j++) { // merge scoreDocs into hq
+        final FieldDoc fieldDoc = (FieldDoc) scoreDocs[j];
+        //it would be so nice if we had a thread-safe insert 
+        lock.lock();
+        try {
+          if (fieldDoc == hq.insertWithOverflow(fieldDoc))
+            break;
+        } finally {
+          lock.unlock();
+        }
+      }
+      return docs;
+    }
+  }
+
+  /**
+   * A helper class that wraps a {@link CompletionService} and provides an
+   * iterable interface to the completed {@link Callable} instances.
+   * 
+   * @param <T>
+   *          the type of the {@link Callable} return value
+   */
+  private static final class ExecutionHelper<T> implements Iterator<T>, Iterable<T> {
+    private final CompletionService<T> service;
+    private int numTasks;
+
+    ExecutionHelper(final Executor executor) {
+      this.service = new ExecutorCompletionService<T>(executor);
+    }
+
+    public boolean hasNext() {
+      return numTasks > 0;
+    }
+
+    public void submit(Callable<T> task) {
+      this.service.submit(task);
+      ++numTasks;
+    }
+
+    public T next() {
+      if(!this.hasNext())
+        throw new NoSuchElementException();
+      try {
+        return service.take().get();
+      } catch (InterruptedException e) {
+        throw new ThreadInterruptedException(e);
+      } catch (ExecutionException e) {
+        throw new RuntimeException(e);
+      } finally {
+        --numTasks;
+      }
+    }
+
+    public void remove() {
+      throw new UnsupportedOperationException();
+    }
+
+    public Iterator<T> iterator() {
+      // use the shortcut here - this is only used in a privat context
+      return this;
+    }
+  }
 }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MatchAllDocsQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MatchAllDocsQuery.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MatchAllDocsQuery.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MatchAllDocsQuery.java Thu Jan 13 02:09:33 2011
@@ -18,8 +18,8 @@ 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.index.MultiFields;
 import org.apache.lucene.util.ToStringUtils;
 import org.apache.lucene.util.Bits;
 
@@ -55,7 +55,7 @@ public class MatchAllDocsQuery extends Q
     MatchAllScorer(IndexReader reader, Similarity similarity, Weight w,
         byte[] norms) throws IOException {
       super(similarity,w);
-      delDocs = MultiFields.getDeletedDocs(reader);
+      delDocs = reader.getDeletedDocs();
       score = w.getValue();
       maxDoc = reader.maxDoc();
       this.norms = norms;
@@ -95,7 +95,7 @@ public class MatchAllDocsQuery extends Q
     private float queryWeight;
     private float queryNorm;
 
-    public MatchAllDocsWeight(Searcher searcher) {
+    public MatchAllDocsWeight(IndexSearcher searcher) {
       this.similarity = searcher.getSimilarity();
     }
 
@@ -127,13 +127,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, boolean scoreDocsInOrder, boolean topScorer) 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:");
@@ -147,7 +147,7 @@ public class MatchAllDocsQuery extends Q
   }
 
   @Override
-  public Weight createWeight(Searcher searcher) {
+  public Weight createWeight(IndexSearcher searcher) {
     return new MatchAllDocsWeight(searcher);
   }
 

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MatchOnlyTermScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MatchOnlyTermScorer.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MatchOnlyTermScorer.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MatchOnlyTermScorer.java Thu Jan 13 02:09:33 2011
@@ -78,7 +78,7 @@ final class MatchOnlyTermScorer extends 
 
   // firstDocID is ignored since nextDoc() sets 'doc'
   @Override
-  protected boolean score(Collector c, int end, int firstDocID) throws IOException {
+  public boolean score(Collector c, int end, int firstDocID) throws IOException {
     c.setScorer(this);
     // nocommit -- this can leave scorer on a deleted doc...
     while (doc < end) {                           // for docs in window

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MaxNonCompetitiveBoostAttribute.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MaxNonCompetitiveBoostAttribute.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MaxNonCompetitiveBoostAttribute.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MaxNonCompetitiveBoostAttribute.java Thu Jan 13 02:09:33 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/bulkpostings/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java Thu Jan 13 02:09:33 2011
@@ -21,6 +21,7 @@ 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;
@@ -134,7 +135,7 @@ public class MultiPhraseQuery extends Qu
     private float queryNorm;
     private float queryWeight;
 
-    public MultiPhraseWeight(Searcher searcher)
+    public MultiPhraseWeight(IndexSearcher searcher)
       throws IOException {
       this.similarity = getSimilarity(searcher);
 
@@ -167,10 +168,10 @@ public class MultiPhraseQuery extends Qu
     }
 
     @Override
-    public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
+    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer) 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 +220,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,7 +233,7 @@ 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:");
@@ -263,7 +264,7 @@ public class MultiPhraseQuery extends Qu
       fieldExpl.setDescription("fieldWeight("+getQuery()+" in "+doc+
                                "), product of:");
 
-      Scorer scorer = scorer(reader, true, false);
+      Scorer scorer = scorer(context, true, false);
       if (scorer == null) {
         return new Explanation(0.0f, "no matching docs");
       }
@@ -283,7 +284,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);
@@ -324,7 +325,7 @@ public class MultiPhraseQuery extends Qu
   }
 
   @Override
-  public Weight createWeight(Searcher searcher) throws IOException {
+  public Weight createWeight(IndexSearcher searcher) throws IOException {
     return new MultiPhraseWeight(searcher);
   }
 

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java Thu Jan 13 02:09:33 2011
@@ -33,7 +33,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.
  *
@@ -201,7 +201,7 @@ public abstract class MultiTermQuery ext
     
     @Override
     protected void addClause(BooleanQuery topLevel, Term term, int docFreq, float boost) {
-      final Query q = new ConstantScoreQuery(new QueryWrapperFilter(new TermQuery(term, docFreq)));
+      final Query q = new ConstantScoreQuery(new TermQuery(term, docFreq));
       q.setBoost(boost);
       topLevel.add(q, BooleanClause.Occur.SHOULD);
     }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java Thu Jan 13 02:09:33 2011
@@ -19,12 +19,12 @@ 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.BulkPostingsEnum;
-import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.util.OpenBitSet;
 import org.apache.lucene.util.Bits;
 
@@ -105,8 +105,9 @@ public class MultiTermQueryWrapperFilter
    * results.
    */
   @Override
-  public DocIdSet getDocIdSet(IndexReader reader) throws IOException {
-    final Fields fields = MultiFields.getFields(reader);
+  public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
+    final IndexReader reader = context.reader;
+    final Fields fields = reader.fields();
     if (fields == null) {
       // reader has no fields
       return DocIdSet.EMPTY_DOCIDSET;
@@ -122,9 +123,9 @@ 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 = MultiFields.getDeletedDocs(reader);
+      final Bits delDocs = reader.getDeletedDocs();
       BulkPostingsEnum postingsEnum = null;
       do {
         termCount++;

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/PhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/PhraseQuery.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/PhraseQuery.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/PhraseQuery.java Thu Jan 13 02:09:33 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;
@@ -143,7 +144,7 @@ public class PhraseQuery extends Query {
     private float queryWeight;
     private IDFExplanation idfExp;
 
-    public PhraseWeight(Searcher searcher)
+    public PhraseWeight(IndexSearcher searcher)
       throws IOException {
       this.similarity = getSimilarity(searcher);
 
@@ -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, boolean scoreDocsInOrder, boolean topScorer) 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, true, false);
       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);
@@ -311,7 +312,7 @@ public class PhraseQuery extends Query {
   }
 
   @Override
-  public Weight createWeight(Searcher searcher) throws IOException {
+  public Weight createWeight(IndexSearcher searcher) throws IOException {
     if (terms.size() == 1) {			  // optimize one-term case
       Term term = terms.get(0);
       Query termQuery = new TermQuery(term);

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Query.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Query.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Query.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Query.java Thu Jan 13 02:09:33 2011
@@ -19,8 +19,6 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
-import java.util.HashSet;
-
 import java.util.Set;
 
 import org.apache.lucene.index.IndexReader;
@@ -89,14 +87,14 @@ public abstract class Query implements j
    * <p>
    * Only implemented by primitive queries, which re-write to themselves.
    */
-  public Weight createWeight(Searcher searcher) throws IOException {
+  public Weight createWeight(IndexSearcher searcher) throws IOException {
     throw new UnsupportedOperationException();
   }
 
   /**
    * Expert: Constructs and initializes a Weight for a top-level query.
    */
-  public Weight weight(Searcher searcher) throws IOException {
+  public Weight weight(IndexSearcher searcher) throws IOException {
     Query query = searcher.rewrite(this);
     Weight weight = query.createWeight(searcher);
     float sum = weight.sumOfSquaredWeights();
@@ -116,52 +114,6 @@ public abstract class Query implements j
     return this;
   }
   
-
-  /** Expert: called when re-writing queries under MultiSearcher.
-   *
-   * Create a single query suitable for use by all subsearchers (in 1-1
-   * correspondence with queries). This is an optimization of the OR of
-   * all queries. We handle the common optimization cases of equal
-   * queries and overlapping clauses of boolean OR queries (as generated
-   * by MultiTermQuery.rewrite()).
-   * Be careful overriding this method as queries[0] determines which
-   * method will be called and is not necessarily of the same type as
-   * the other queries.
-  */
-  public Query combine(Query[] queries) {
-    HashSet<Query> uniques = new HashSet<Query>();
-    for (int i = 0; i < queries.length; i++) {
-      Query query = queries[i];
-      BooleanClause[] clauses = null;
-      // check if we can split the query into clauses
-      boolean splittable = (query instanceof BooleanQuery);
-      if(splittable){
-        BooleanQuery bq = (BooleanQuery) query;
-        splittable = bq.isCoordDisabled();
-        clauses = bq.getClauses();
-        for (int j = 0; splittable && j < clauses.length; j++) {
-          splittable = (clauses[j].getOccur() == BooleanClause.Occur.SHOULD);
-        }
-      }
-      if(splittable){
-        for (int j = 0; j < clauses.length; j++) {
-          uniques.add(clauses[j].getQuery());
-        }
-      } else {
-        uniques.add(query);
-      }
-    }
-    // optimization: if we have just one query, just return it
-    if(uniques.size() == 1){
-        return uniques.iterator().next();
-    }
-    BooleanQuery result = new BooleanQuery(true);
-    for (final Query query : uniques)
-      result.add(query, BooleanClause.Occur.SHOULD);
-    return result;
-  }
-  
-
   /**
    * Expert: adds all terms occurring in this query to the terms set. Only
    * works if this query is in its {@link #rewrite rewritten} form.
@@ -174,35 +126,11 @@ public abstract class Query implements j
   }
   
 
-
-  /** Expert: merges the clauses of a set of BooleanQuery's into a single
-   * BooleanQuery.
-   *
-   *<p>A utility for use by {@link #combine(Query[])} implementations.
-   */
-  public static Query mergeBooleanQueries(BooleanQuery... queries) {
-    HashSet<BooleanClause> allClauses = new HashSet<BooleanClause>();
-    for (BooleanQuery booleanQuery : queries) {
-      for (BooleanClause clause : booleanQuery) {
-        allClauses.add(clause);
-      }
-    }
-
-    boolean coordDisabled =
-      queries.length==0? false : queries[0].isCoordDisabled();
-    BooleanQuery result = new BooleanQuery(coordDisabled);
-    for(BooleanClause clause2 : allClauses) {
-      result.add(clause2);
-    }
-    return result;
-  }
-  
-
   /** 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(Searcher searcher) {
+  public Similarity getSimilarity(IndexSearcher searcher) {
     return searcher.getSimilarity();
   }