You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2011/11/03 14:35:20 UTC

svn commit: r1197110 [4/10] - in /lucene/dev/branches/solrcloud: ./ dev-tools/eclipse/ dev-tools/idea/.idea/ dev-tools/idea/.idea/libraries/ dev-tools/idea/lucene/contrib/ dev-tools/idea/modules/benchmark/ dev-tools/idea/solr/contrib/langid/ dev-tools/...

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java?rev=1197110&r1=1197109&r2=1197110&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java Thu Nov  3 13:35:07 2011
@@ -19,17 +19,22 @@ package org.apache.lucene.index.values;
 
 import java.io.IOException;
 import java.util.Comparator;
+import java.util.List;
 
-import org.apache.lucene.index.values.Bytes.BytesSortedSourceBase;
+import org.apache.lucene.index.codecs.MergeState;
 import org.apache.lucene.index.values.Bytes.BytesReaderBase;
+import org.apache.lucene.index.values.Bytes.BytesSortedSourceBase;
 import org.apache.lucene.index.values.Bytes.DerefBytesWriterBase;
 import org.apache.lucene.index.values.IndexDocValues.SortedSource;
+import org.apache.lucene.index.values.SortedBytesMergeUtils.MergeContext;
+import org.apache.lucene.index.values.SortedBytesMergeUtils.SortedSourceSlice;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.Counter;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.packed.PackedInts;
 
 // Stores fixed-length byte[] by deref, ie when two docs
@@ -53,6 +58,37 @@ class FixedSortedBytesImpl {
       this.comp = comp;
     }
 
+    @Override
+    public void merge(MergeState mergeState, IndexDocValues[] docValues)
+        throws IOException {
+      boolean success = false;
+      try {
+        final MergeContext ctx = SortedBytesMergeUtils.init(ValueType.BYTES_FIXED_SORTED, docValues, comp, mergeState);
+        List<SortedSourceSlice> slices = SortedBytesMergeUtils.buildSlices(mergeState, docValues, ctx);
+        final IndexOutput datOut = getOrCreateDataOut();
+        datOut.writeInt(ctx.sizePerValues);
+        final int maxOrd = SortedBytesMergeUtils.mergeRecords(ctx, datOut, slices);
+        
+        final IndexOutput idxOut = getOrCreateIndexOut();
+        idxOut.writeInt(maxOrd);
+        final PackedInts.Writer ordsWriter = PackedInts.getWriter(idxOut, ctx.docToEntry.length,
+            PackedInts.bitsRequired(maxOrd));
+        for (SortedSourceSlice slice : slices) {
+          slice.writeOrds(ordsWriter);
+        }
+        ordsWriter.finish();
+        success = true;
+      } finally {
+        releaseResources();
+        if (success) {
+          IOUtils.close(getIndexOut(), getDataOut());
+        } else {
+          IOUtils.closeWhileHandlingException(getIndexOut(), getDataOut());
+        }
+
+      }
+    }
+
     // Important that we get docCount, in case there were
     // some last docs that we didn't see
     @Override
@@ -60,15 +96,15 @@ class FixedSortedBytesImpl {
       fillDefault(docCount);
       final IndexOutput datOut = getOrCreateDataOut();
       final int count = hash.size();
-      final int[] address = new int[count]; // addr 0 is default values
+      final int[] address = new int[count];
       datOut.writeInt(size);
       if (size != -1) {
         final int[] sortedEntries = hash.sort(comp);
         // first dump bytes data, recording address as we go
-        final BytesRef bytesRef = new BytesRef(size);
+        final BytesRef spare = new BytesRef(size);
         for (int i = 0; i < count; i++) {
           final int e = sortedEntries[i];
-          final BytesRef bytes = hash.get(e, bytesRef);
+          final BytesRef bytes = hash.get(e, spare);
           assert bytes.length == size;
           datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
           address[e] = i;
@@ -95,8 +131,8 @@ class FixedSortedBytesImpl {
 
     @Override
     public Source load() throws IOException {
-      return new FixedSortedSource(cloneData(), cloneIndex(), size,
-          valueCount, comparator);
+      return new FixedSortedSource(cloneData(), cloneIndex(), size, valueCount,
+          comparator);
     }
 
     @Override
@@ -104,7 +140,7 @@ class FixedSortedBytesImpl {
       return new DirectFixedSortedSource(cloneData(), cloneIndex(), size,
           valueCount, comparator, type);
     }
-    
+
     @Override
     public int getValueSize() {
       return size;
@@ -117,7 +153,8 @@ class FixedSortedBytesImpl {
 
     FixedSortedSource(IndexInput datIn, IndexInput idxIn, int size,
         int numValues, Comparator<BytesRef> comp) throws IOException {
-      super(datIn, idxIn, comp, size * numValues, ValueType.BYTES_FIXED_SORTED);
+      super(datIn, idxIn, comp, size * numValues, ValueType.BYTES_FIXED_SORTED,
+          false);
       this.size = size;
       this.valueCount = numValues;
       closeIndexInput();
@@ -165,9 +202,7 @@ class FixedSortedBytesImpl {
     public BytesRef getByOrd(int ord, BytesRef bytesRef) {
       try {
         datIn.seek(basePointer + size * ord);
-        if (bytesRef.bytes.length < size) {
-          bytesRef.grow(size);
-        }
+        bytesRef.grow(size);
         datIn.readBytes(bytesRef.bytes, 0, size);
         bytesRef.length = size;
         bytesRef.offset = 0;
@@ -182,4 +217,5 @@ class FixedSortedBytesImpl {
       return valueCount;
     }
   }
+
 }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/values/IndexDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/values/IndexDocValues.java?rev=1197110&r1=1197109&r2=1197110&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/values/IndexDocValues.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/values/IndexDocValues.java Thu Nov  3 13:35:07 2011
@@ -278,7 +278,7 @@ public abstract class IndexDocValues imp
       return binarySearch(value, spare, 0, getValueCount() - 1);
     }    
 
-    protected int binarySearch(BytesRef b, BytesRef bytesRef, int low,
+    private int binarySearch(BytesRef b, BytesRef bytesRef, int low,
         int high) {
       int mid = 0;
       while (low <= high) {

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java?rev=1197110&r1=1197109&r2=1197110&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java Thu Nov  3 13:35:07 2011
@@ -19,17 +19,22 @@ package org.apache.lucene.index.values;
 
 import java.io.IOException;
 import java.util.Comparator;
+import java.util.List;
 
+import org.apache.lucene.index.codecs.MergeState;
 import org.apache.lucene.index.values.Bytes.BytesSortedSourceBase;
 import org.apache.lucene.index.values.Bytes.BytesReaderBase;
 import org.apache.lucene.index.values.Bytes.DerefBytesWriterBase;
 import org.apache.lucene.index.values.IndexDocValues.SortedSource;
+import org.apache.lucene.index.values.SortedBytesMergeUtils.MergeContext;
+import org.apache.lucene.index.values.SortedBytesMergeUtils.SortedSourceSlice;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.Counter;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.packed.PackedInts;
 
 // Stores variable-length byte[] by deref, ie when two docs
@@ -54,6 +59,47 @@ final class VarSortedBytesImpl {
       this.comp = comp;
       size = 0;
     }
+    @Override
+    public void merge(MergeState mergeState, IndexDocValues[] docValues)
+        throws IOException {
+      boolean success = false;
+      try {
+        MergeContext ctx = SortedBytesMergeUtils.init(ValueType.BYTES_VAR_SORTED, docValues, comp, mergeState);
+        final List<SortedSourceSlice> slices = SortedBytesMergeUtils.buildSlices(mergeState, docValues, ctx);
+        IndexOutput datOut = getOrCreateDataOut();
+        
+        ctx.offsets = new long[1];
+        final int maxOrd = SortedBytesMergeUtils.mergeRecords(ctx, datOut, slices);
+        final long[] offsets = ctx.offsets;
+        maxBytes = offsets[maxOrd-1];
+        final IndexOutput idxOut = getOrCreateIndexOut();
+        
+        idxOut.writeLong(maxBytes);
+        final PackedInts.Writer offsetWriter = PackedInts.getWriter(idxOut, maxOrd+1,
+            PackedInts.bitsRequired(maxBytes));
+        offsetWriter.add(0);
+        for (int i = 0; i < maxOrd; i++) {
+          offsetWriter.add(offsets[i]);
+        }
+        offsetWriter.finish();
+        
+        final PackedInts.Writer ordsWriter = PackedInts.getWriter(idxOut, ctx.docToEntry.length,
+            PackedInts.bitsRequired(maxOrd-1));
+        for (SortedSourceSlice slice : slices) {
+          slice.writeOrds(ordsWriter);
+        }
+        ordsWriter.finish();
+        success = true;
+      } finally {
+        releaseResources();
+        if (success) {
+          IOUtils.close(getIndexOut(), getDataOut());
+        } else {
+          IOUtils.closeWhileHandlingException(getIndexOut(), getDataOut());
+        }
+
+      }
+    }
 
     @Override
     protected void checkSize(BytesRef bytes) {
@@ -67,35 +113,31 @@ final class VarSortedBytesImpl {
       fillDefault(docCount);
       final int count = hash.size();
       final IndexOutput datOut = getOrCreateDataOut();
+      final IndexOutput idxOut = getOrCreateIndexOut();
       long offset = 0;
       final int[] index = new int[count];
-      final long[] offsets = new long[count];
       final int[] sortedEntries = hash.sort(comp);
-      // first dump bytes data, recording index & offset as
+      // total bytes of data
+      idxOut.writeLong(maxBytes);
+      PackedInts.Writer offsetWriter = PackedInts.getWriter(idxOut, count+1,
+          PackedInts.bitsRequired(maxBytes));
+      // first dump bytes data, recording index & write offset as
       // we go
+      final BytesRef spare = new BytesRef();
       for (int i = 0; i < count; i++) {
         final int e = sortedEntries[i];
-        offsets[i] = offset;
+        offsetWriter.add(offset);
         index[e] = i;
-
-        final BytesRef bytes = hash.get(e, new BytesRef());
+        final BytesRef bytes = hash.get(e, spare);
         // TODO: we could prefix code...
         datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
         offset += bytes.length;
       }
-      final IndexOutput idxOut = getOrCreateIndexOut();
-      // total bytes of data
-      idxOut.writeLong(offset);
-      // write index
-      writeIndex(idxOut, docCount, count, index, docToEntry);
-      // next ord (0-based) -> offset
-      PackedInts.Writer offsetWriter = PackedInts.getWriter(idxOut, count+1,
-          PackedInts.bitsRequired(offset));
-      for (int i = 0; i < count; i++) {
-        offsetWriter.add(offsets[i]);
-      }
       offsetWriter.add(offset);
       offsetWriter.finish();
+      // write index
+      writeIndex(idxOut, docCount, count, index, docToEntry);
+
     }
   }
 
@@ -123,13 +165,11 @@ final class VarSortedBytesImpl {
     
   }
   private static final class VarSortedSource extends BytesSortedSourceBase {
-    private final PackedInts.Reader ordToOffsetIndex; // 0-based
     private final int valueCount;
 
     VarSortedSource(IndexInput datIn, IndexInput idxIn,
         Comparator<BytesRef> comp) throws IOException {
-      super(datIn, idxIn, comp, idxIn.readLong(), ValueType.BYTES_VAR_SORTED);
-      ordToOffsetIndex = PackedInts.getReader(idxIn);
+      super(datIn, idxIn, comp, idxIn.readLong(), ValueType.BYTES_VAR_SORTED, true);
       valueCount = ordToOffsetIndex.size()-1; // the last value here is just a dummy value to get the length of the last value
       closeIndexInput();
     }
@@ -149,7 +189,7 @@ final class VarSortedBytesImpl {
   }
 
   private static final class DirectSortedSource extends SortedSource {
-    private final PackedInts.Reader docToOrdIndex;
+    private final PackedInts.RandomAccessReaderIterator docToOrdIndex;
     private final PackedInts.RandomAccessReaderIterator ordToOffsetIndex;
     private final IndexInput datIn;
     private final long basePointer;
@@ -159,16 +199,22 @@ final class VarSortedBytesImpl {
         Comparator<BytesRef> comparator, ValueType type) throws IOException {
       super(type, comparator);
       idxIn.readLong();
-      docToOrdIndex = PackedInts.getReader(idxIn); // read the ords in to prevent too many random disk seeks
       ordToOffsetIndex = PackedInts.getRandomAccessReaderIterator(idxIn);
       valueCount = ordToOffsetIndex.size()-1; // the last value here is just a dummy value to get the length of the last value
+      // advance this iterator to the end and clone the stream once it points to the docToOrdIndex header
+      ordToOffsetIndex.advance(valueCount);
+      docToOrdIndex = PackedInts.getRandomAccessReaderIterator((IndexInput) idxIn.clone()); // read the ords in to prevent too many random disk seeks
       basePointer = datIn.getFilePointer();
       this.datIn = datIn;
     }
 
     @Override
     public int ord(int docID) {
-      return (int) docToOrdIndex.get(docID);
+      try {
+        return (int) docToOrdIndex.get(docID);
+      } catch (IOException ex) {
+        throw new IllegalStateException("failed", ex);
+      }
     }
 
     @Override
@@ -178,9 +224,7 @@ final class VarSortedBytesImpl {
         final long nextOffset = ordToOffsetIndex.next();
         datIn.seek(basePointer + offset);
         final int length = (int) (nextOffset - offset);
-        if (bytesRef.bytes.length < length) {
-          bytesRef.grow(length);
-        }
+        bytesRef.grow(length);
         datIn.readBytes(bytesRef.bytes, 0, length);
         bytesRef.length = length;
         bytesRef.offset = 0;

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/BooleanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/BooleanQuery.java?rev=1197110&r1=1197109&r2=1197110&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/BooleanQuery.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/BooleanQuery.java Thu Nov  3 13:35:07 2011
@@ -329,7 +329,7 @@ public class BooleanQuery extends Query 
       }
       
       // Check if we can return a BooleanScorer
-      if (!scoreDocsInOrder && topScorer && required.size() == 0 && prohibited.size() < 32) {
+      if (!scoreDocsInOrder && topScorer && required.size() == 0) {
         return new BooleanScorer(this, disableCoord, minNrShouldMatch, optional, prohibited, maxCoord);
       }
       
@@ -366,19 +366,12 @@ public class BooleanQuery extends Query 
     
     @Override
     public boolean scoresDocsOutOfOrder() {
-      int numProhibited = 0;
       for (BooleanClause c : clauses) {
         if (c.isRequired()) {
           return false; // BS2 (in-order) will be used by scorer()
-        } else if (c.isProhibited()) {
-          ++numProhibited;
         }
       }
       
-      if (numProhibited > 32) { // cannot use BS
-        return false;
-      }
-      
       // scorer() will return an out-of-order scorer if requested.
       return true;
     }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/BooleanScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/BooleanScorer.java?rev=1197110&r1=1197109&r2=1197110&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/BooleanScorer.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/BooleanScorer.java Thu Nov  3 13:35:07 2011
@@ -29,10 +29,10 @@ import org.apache.lucene.search.BooleanQ
 /* Description from Doug Cutting (excerpted from
  * LUCENE-1483):
  *
- * BooleanScorer uses a ~16k array to score windows of
- * docs. So it scores docs 0-16k first, then docs 16-32k,
+ * BooleanScorer uses an array to score windows of
+ * 2K docs. So it scores docs 0-2K first, then docs 2K-4K,
  * etc. For each window it iterates through all query terms
- * and accumulates a score in table[doc%16k]. It also stores
+ * and accumulates a score in table[doc%2K]. It also stores
  * in the table a bitmask representing which terms
  * contributed to the score. Non-zero scores are chained in
  * a linked list. At the end of scoring each window it then
@@ -75,9 +75,7 @@ final class BooleanScorer extends Scorer
     public void collect(final int doc) throws IOException {
       final BucketTable table = bucketTable;
       final int i = doc & BucketTable.MASK;
-      Bucket bucket = table.buckets[i];
-      if (bucket == null)
-        table.buckets[i] = bucket = new Bucket();
+      final Bucket bucket = table.buckets[i];
       
       if (bucket.doc != doc) {                    // invalid bucket
         bucket.doc = doc;                         // set doc
@@ -143,6 +141,9 @@ final class BooleanScorer extends Scorer
   static final class Bucket {
     int doc = -1;            // tells if bucket is valid
     float score;             // incremental score
+    // TODO: break out bool anyProhibited, int
+    // numRequiredMatched; then we can remove 32 limit on
+    // required clauses
     int bits;                // used for bool constraints
     int coord;               // count of terms in score
     Bucket next;             // next valid bucket
@@ -156,7 +157,13 @@ final class BooleanScorer extends Scorer
     final Bucket[] buckets = new Bucket[SIZE];
     Bucket first = null;                          // head of valid list
   
-    public BucketTable() {}
+    public BucketTable() {
+      // Pre-fill to save the lazy init when collecting
+      // each sub:
+      for(int idx=0;idx<SIZE;idx++) {
+        buckets[idx] = new Bucket();
+      }
+    }
 
     public Collector newCollector(int mask) {
       return new BooleanScorerCollector(mask, this);
@@ -169,7 +176,7 @@ final class BooleanScorer extends Scorer
     public Scorer scorer;
     // TODO: re-enable this if BQ ever sends us required clauses
     //public boolean required = false;
-    public boolean prohibited = false;
+    public boolean prohibited;
     public Collector collector;
     public SubScorer next;
 
@@ -193,12 +200,13 @@ final class BooleanScorer extends Scorer
   private final float[] coordFactors;
   // TODO: re-enable this if BQ ever sends us required clauses
   //private int requiredMask = 0;
-  private int prohibitedMask = 0;
-  private int nextMask = 1;
   private final int minNrShouldMatch;
   private int end;
   private Bucket current;
   private int doc = -1;
+
+  // Any time a prohibited clause matches we set bit 0:
+  private static final int PROHIBITED_MASK = 1;
   
   BooleanScorer(BooleanWeight weight, boolean disableCoord, int minNrShouldMatch,
       List<Scorer> optionalScorers, List<Scorer> prohibitedScorers, int maxCoord) throws IOException {
@@ -215,11 +223,8 @@ final class BooleanScorer extends Scorer
     
     if (prohibitedScorers != null && prohibitedScorers.size() > 0) {
       for (Scorer scorer : prohibitedScorers) {
-        int mask = nextMask;
-        nextMask = nextMask << 1;
-        prohibitedMask |= mask;                     // update prohibited mask
         if (scorer.nextDoc() != NO_MORE_DOCS) {
-          scorers = new SubScorer(scorer, false, true, bucketTable.newCollector(mask), scorers);
+          scorers = new SubScorer(scorer, false, true, bucketTable.newCollector(PROHIBITED_MASK), scorers);
         }
       }
     }
@@ -233,9 +238,12 @@ final class BooleanScorer extends Scorer
   // firstDocID is ignored since nextDoc() initializes 'current'
   @Override
   public boolean score(Collector collector, int max, int firstDocID) throws IOException {
+    // Make sure it's only BooleanScorer that calls us:
+    assert firstDocID == -1;
     boolean more;
     Bucket tmp;
     BucketScorer bs = new BucketScorer(weight);
+
     // The internal loop will set the score and doc before calling collect.
     collector.setScorer(bs);
     do {
@@ -244,12 +252,13 @@ final class BooleanScorer extends Scorer
       while (current != null) {         // more queued 
 
         // check prohibited & required
-        if ((current.bits & prohibitedMask) == 0) {
+        if ((current.bits & PROHIBITED_MASK) == 0) {
 
-            // TODO: re-enable this if BQ ever sends us required
-            // clauses
-            //&& (current.bits & requiredMask) == requiredMask) {
+          // TODO: re-enable this if BQ ever sends us required
+          // clauses
+          //&& (current.bits & requiredMask) == requiredMask) {
           
+          // TODO: can we remove this?  
           if (current.doc >= max){
             tmp = current;
             current = current.next;
@@ -298,48 +307,22 @@ final class BooleanScorer extends Scorer
 
   @Override
   public int docID() {
-    return doc;
+    throw new UnsupportedOperationException();
   }
 
   @Override
   public int nextDoc() throws IOException {
-    boolean more;
-    do {
-      while (bucketTable.first != null) {         // more queued
-        current = bucketTable.first;
-        bucketTable.first = current.next;         // pop the queue
-
-        // check prohibited & required, and minNrShouldMatch
-        if ((current.bits & prohibitedMask) == 0 &&
-            current.coord >= minNrShouldMatch) {
-          // TODO: re-enable this if BQ ever sends us required clauses
-          // (current.bits & requiredMask) == requiredMask &&
-          return doc = current.doc;
-        }
-      }
-
-      // refill the queue
-      more = false;
-      end += BucketTable.SIZE;
-      for (SubScorer sub = scorers; sub != null; sub = sub.next) {
-        int subScorerDocID = sub.scorer.docID();
-        if (subScorerDocID != NO_MORE_DOCS) {
-          more |= sub.scorer.score(sub.collector, end, subScorerDocID);
-        }
-      }
-    } while (bucketTable.first != null || more);
-
-    return doc = NO_MORE_DOCS;
+    throw new UnsupportedOperationException();
   }
 
   @Override
   public float score() {
-    return current.score * coordFactors[current.coord];
+    throw new UnsupportedOperationException();
   }
 
   @Override
   public void score(Collector collector) throws IOException {
-    score(collector, Integer.MAX_VALUE, nextDoc());
+    score(collector, Integer.MAX_VALUE, -1);
   }
   
   @Override

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/CachingSpanFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/CachingSpanFilter.java?rev=1197110&r1=1197109&r2=1197110&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/CachingSpanFilter.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/CachingSpanFilter.java Thu Nov  3 13:35:07 2011
@@ -19,8 +19,11 @@ package org.apache.lucene.search;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.FixedBitSet;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 
 /**
  * Wraps another SpanFilter's result and caches it.  The purpose is to allow
@@ -34,36 +37,63 @@ public class CachingSpanFilter extends S
    */
   private final CachingWrapperFilter.FilterCache<SpanFilterResult> cache;
 
-  /**
-   * New deletions always result in a cache miss, by default
-   * ({@link CachingWrapperFilter.DeletesMode#RECACHE}.
+  /** Wraps another SpanFilter's result and caches it.
    * @param filter Filter to cache results of
    */
   public CachingSpanFilter(SpanFilter filter) {
-    this(filter, CachingWrapperFilter.DeletesMode.RECACHE);
+    this.filter = filter;
+    this.cache = new CachingWrapperFilter.FilterCache<SpanFilterResult>();
   }
 
-  /**
-   * @param filter Filter to cache results of
-   * @param deletesMode See {@link CachingWrapperFilter.DeletesMode}
-   */
-  public CachingSpanFilter(SpanFilter filter, CachingWrapperFilter.DeletesMode deletesMode) {
-    this.filter = filter;
-    if (deletesMode == CachingWrapperFilter.DeletesMode.DYNAMIC) {
-      throw new IllegalArgumentException("DeletesMode.DYNAMIC is not supported");
-    }
-    this.cache = new CachingWrapperFilter.FilterCache<SpanFilterResult>(deletesMode) {
-      @Override
-      protected SpanFilterResult mergeLiveDocs(final Bits liveDocs, final SpanFilterResult value) {
-        throw new IllegalStateException("DeletesMode.DYNAMIC is not supported");
-      }
-    };
+  @Override
+  public DocIdSet getDocIdSet(AtomicReaderContext context, final Bits acceptDocs) throws IOException {
+    final SpanFilterResult result = getCachedResult(context);
+    return BitsFilteredDocIdSet.wrap(result.getDocIdSet(), acceptDocs);
   }
 
   @Override
-  public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
-    SpanFilterResult result = getCachedResult(context);
-    return result != null ? result.getDocIdSet() : null;
+  public SpanFilterResult bitSpans(AtomicReaderContext context, final Bits acceptDocs) throws IOException {
+    final SpanFilterResult result = getCachedResult(context);
+    if (acceptDocs == null) {
+      return result;
+    } else {
+      // TODO: filter positions more efficient
+      List<SpanFilterResult.PositionInfo> allPositions = result.getPositions();
+      List<SpanFilterResult.PositionInfo> positions = new ArrayList<SpanFilterResult.PositionInfo>(allPositions.size() / 2 + 1);
+      for (SpanFilterResult.PositionInfo p : allPositions) {
+        if (acceptDocs.get(p.getDoc())) {
+          positions.add(p);
+        }        
+      }
+      return new SpanFilterResult(BitsFilteredDocIdSet.wrap(result.getDocIdSet(), acceptDocs), positions);
+    }
+  }
+
+  /** Provide the DocIdSet to be cached, using the DocIdSet provided
+   *  by the wrapped Filter.
+   *  <p>This implementation returns the given {@link DocIdSet}, if {@link DocIdSet#isCacheable}
+   *  returns <code>true</code>, else it copies the {@link DocIdSetIterator} into
+   *  an {@link FixedBitSet}.
+   */
+  protected SpanFilterResult spanFilterResultToCache(SpanFilterResult result, IndexReader reader) throws IOException {
+    if (result == null || result.getDocIdSet() == null) {
+      // this is better than returning null, as the nonnull result can be cached
+      return SpanFilterResult.EMPTY_SPAN_FILTER_RESULT;
+    } else if (result.getDocIdSet().isCacheable()) {
+      return result;
+    } else {
+      final DocIdSetIterator it = result.getDocIdSet().iterator();
+      // null is allowed to be returned by iterator(),
+      // in this case we wrap with the empty set,
+      // which is cacheable.
+      if (it == null) {
+        return SpanFilterResult.EMPTY_SPAN_FILTER_RESULT;
+      } else {
+        final FixedBitSet bits = new FixedBitSet(reader.maxDoc());
+        bits.or(it);
+        return new SpanFilterResult(bits, result.getPositions());
+      }
+    }
   }
   
   // for testing
@@ -71,29 +101,23 @@ public class CachingSpanFilter extends S
 
   private SpanFilterResult getCachedResult(AtomicReaderContext context) throws IOException {
     final IndexReader reader = context.reader;
-
     final Object coreKey = reader.getCoreCacheKey();
-    final Object delCoreKey = reader.hasDeletions() ? reader.getLiveDocs() : coreKey;
 
-    SpanFilterResult result = cache.get(reader, coreKey, delCoreKey);
+    SpanFilterResult result = cache.get(reader, coreKey);
     if (result != null) {
       hitCount++;
       return result;
+    } else {
+      missCount++;
+      // cache miss: we use no acceptDocs here
+      // (this saves time on building SpanFilterResult, the acceptDocs will be applied on the cached set)
+      result = spanFilterResultToCache(filter.bitSpans(context, null/**!!!*/), reader);
+      cache.put(coreKey, result);
     }
-
-    missCount++;
-    result = filter.bitSpans(context);
-
-    cache.put(coreKey, delCoreKey, result);
+    
     return result;
   }
 
-
-  @Override
-  public SpanFilterResult bitSpans(AtomicReaderContext context) throws IOException {
-    return getCachedResult(context);
-  }
-
   @Override
   public String toString() {
     return "CachingSpanFilter("+filter+")";

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/CachingWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/CachingWrapperFilter.java?rev=1197110&r1=1197109&r2=1197110&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/CachingWrapperFilter.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/CachingWrapperFilter.java Thu Nov  3 13:35:07 2011
@@ -30,11 +30,6 @@ import org.apache.lucene.util.Bits;
  * Wraps another filter's result and caches it.  The purpose is to allow
  * filters to simply filter, and then wrap with this class
  * to add caching.
- *
- * <p><b>NOTE</b>: if you wrap this filter as a query (eg,
- * using ConstantScoreQuery), you'll likely want to enforce
- * deletions (using either {@link DeletesMode#RECACHE} or
- * {@link DeletesMode#DYNAMIC}).
  */
 public class CachingWrapperFilter extends Filter {
   // TODO: make this filter aware of ReaderContext. a cached filter could 
@@ -42,32 +37,9 @@ public class CachingWrapperFilter extend
   // level of the readers hierarchy it should be cached.
   Filter filter;
 
-  /**
-   * Expert: Specifies how new deletions against a reopened
-   * reader should be handled.
-   *
-   * <p>The default is IGNORE, which means the cache entry
-   * will be re-used for a given segment, even when that
-   * segment has been reopened due to changes in deletions.
-   * This is a big performance gain, especially with
-   * near-real-timer readers, since you don't hit a cache
-   * miss on every reopened reader for prior segments.</p>
-   *
-   * <p>However, in some cases this can cause invalid query
-   * results, allowing deleted documents to be returned.
-   * This only happens if the main query does not rule out
-   * deleted documents on its own, such as a toplevel
-   * ConstantScoreQuery.  To fix this, use RECACHE to
-   * re-create the cached filter (at a higher per-reopen
-   * cost, but at faster subsequent search performance), or
-   * use DYNAMIC to dynamically intersect deleted docs (fast
-   * reopen time but some hit to search performance).</p>
-   */
-  public static enum DeletesMode {IGNORE, RECACHE, DYNAMIC};
-
   protected final FilterCache<DocIdSet> cache;
 
-  static abstract class FilterCache<T> {
+  static class FilterCache<T> {
 
     /**
      * A transient Filter cache (package private because of test)
@@ -76,97 +48,27 @@ public class CachingWrapperFilter extend
     // after de-serialize
     transient Map<Object,T> cache;
 
-    private final DeletesMode deletesMode;
-
-    public FilterCache(DeletesMode deletesMode) {
-      this.deletesMode = deletesMode;
-    }
-
-    public synchronized T get(IndexReader reader, Object coreKey, Object delCoreKey) throws IOException {
+    public synchronized T get(IndexReader reader, Object coreKey) throws IOException {
       T value;
 
       if (cache == null) {
         cache = new WeakHashMap<Object,T>();
       }
 
-      if (deletesMode == DeletesMode.IGNORE) {
-        // key on core
-        value = cache.get(coreKey);
-      } else if (deletesMode == DeletesMode.RECACHE) {
-        // key on deletes, if any, else core
-        value = cache.get(delCoreKey);
-      } else {
-
-        assert deletesMode == DeletesMode.DYNAMIC;
-
-        // first try for exact match
-        value = cache.get(delCoreKey);
-
-        if (value == null) {
-          // now for core match, but dynamically AND
-          // live docs
-          value = cache.get(coreKey);
-          if (value != null) {
-            final Bits liveDocs = reader.getLiveDocs();
-            if (liveDocs != null) {
-              value = mergeLiveDocs(liveDocs, value);
-            }
-          }
-        }
-      }
-
-      return value;
+      return cache.get(coreKey);
     }
 
-    protected abstract T mergeLiveDocs(Bits liveDocs, T value);
-
-    public synchronized void put(Object coreKey, Object delCoreKey, T value) {
-      if (deletesMode == DeletesMode.IGNORE) {
-        cache.put(coreKey, value);
-      } else if (deletesMode == DeletesMode.RECACHE) {
-        cache.put(delCoreKey, value);
-      } else {
-        cache.put(coreKey, value);
-        cache.put(delCoreKey, value);
-      }
+    public synchronized void put(Object coreKey, T value) {
+      cache.put(coreKey, value);
     }
   }
 
-  /**
-   * New deletes are ignored by default, which gives higher
-   * cache hit rate on reopened readers.  Most of the time
-   * this is safe, because the filter will be AND'd with a
-   * Query that fully enforces deletions.  If instead you
-   * need this filter to always enforce deletions, pass
-   * either {@link DeletesMode#RECACHE} or {@link
-   * DeletesMode#DYNAMIC}.
+  /** Wraps another filter's result and caches it.
    * @param filter Filter to cache results of
    */
   public CachingWrapperFilter(Filter filter) {
-    this(filter, DeletesMode.IGNORE);
-  }
-
-  /**
-   * Expert: by default, the cached filter will be shared
-   * across reopened segments that only had changes to their
-   * deletions.  
-   *
-   * @param filter Filter to cache results of
-   * @param deletesMode See {@link DeletesMode}
-   */
-  public CachingWrapperFilter(Filter filter, DeletesMode deletesMode) {
     this.filter = filter;
-    cache = new FilterCache<DocIdSet>(deletesMode) {
-      @Override
-      public DocIdSet mergeLiveDocs(final Bits liveDocs, final DocIdSet docIdSet) {
-        return new FilteredDocIdSet(docIdSet) {
-          @Override
-          protected boolean match(int docID) {
-            return liveDocs.get(docID);
-          }
-        };
-      }
-    };
+    cache = new FilterCache<DocIdSet>();
   }
 
   /** Provide the DocIdSet to be cached, using the DocIdSet provided
@@ -200,27 +102,22 @@ public class CachingWrapperFilter extend
   int hitCount, missCount;
 
   @Override
-  public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
+  public DocIdSet getDocIdSet(AtomicReaderContext context, final Bits acceptDocs) throws IOException {
     final IndexReader reader = context.reader;
     final Object coreKey = reader.getCoreCacheKey();
-    final Object delCoreKey = reader.hasDeletions() ? reader.getLiveDocs() : coreKey;
 
-    DocIdSet docIdSet = cache.get(reader, coreKey, delCoreKey);
+    DocIdSet docIdSet = cache.get(reader, coreKey);
     if (docIdSet != null) {
       hitCount++;
-      return docIdSet;
-    }
-
-    missCount++;
-
-    // cache miss
-    docIdSet = docIdSetToCache(filter.getDocIdSet(context), reader);
-
-    if (docIdSet != null) {
-      cache.put(coreKey, delCoreKey, docIdSet);
+    } else {
+      missCount++;
+      // cache miss: we use no acceptDocs here
+      // (this saves time on building DocIdSet, the acceptDocs will be applied on the cached set)
+      docIdSet = docIdSetToCache(filter.getDocIdSet(context, null/**!!!*/), reader);
+      cache.put(coreKey, docIdSet);
     }
     
-    return docIdSet;
+    return BitsFilteredDocIdSet.wrap(docIdSet, acceptDocs);
   }
 
   @Override

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java?rev=1197110&r1=1197109&r2=1197110&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java Thu Nov  3 13:35:07 2011
@@ -30,12 +30,6 @@ import java.util.Set;
  * 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
- * enforce deletions in the filter (using either {@link
- * CachingWrapperFilter.DeletesMode#RECACHE} or {@link
- * CachingWrapperFilter.DeletesMode#DYNAMIC}).
  */
 public class ConstantScoreQuery extends Query {
   protected final Filter filter;
@@ -128,11 +122,11 @@ public class ConstantScoreQuery extends 
 
     @Override
     public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
-        boolean topScorer, Bits acceptDocs) throws IOException {
+        boolean topScorer, final Bits acceptDocs) throws IOException {
       final DocIdSetIterator disi;
       if (filter != null) {
         assert query == null;
-        final DocIdSet dis = filter.getDocIdSet(context);
+        final DocIdSet dis = filter.getDocIdSet(context, acceptDocs);
         if (dis == null) {
           return null;
         }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/DocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/DocIdSet.java?rev=1197110&r1=1197109&r2=1197110&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/DocIdSet.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/DocIdSet.java Thu Nov  3 13:35:07 2011
@@ -18,6 +18,7 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
+import org.apache.lucene.util.Bits;
 
 /**
  * A DocIdSet contains a set of doc ids. Implementing classes must
@@ -46,6 +47,12 @@ public abstract class DocIdSet {
     public boolean isCacheable() {
       return true;
     }
+    
+    // we explicitely provide no random access, as this filter is 100% sparse and iterator exits faster
+    @Override
+    public Bits bits() throws IOException {
+      return null;
+    }
   };
     
   /** Provides a {@link DocIdSetIterator} to access the set.
@@ -54,6 +61,23 @@ public abstract class DocIdSet {
    * are no docs that match. */
   public abstract DocIdSetIterator iterator() throws IOException;
 
+  /** Optionally provides a {@link Bits} interface for random access
+   * to matching documents.
+   * @return {@code null}, if this {@code DocIdSet} does not support random access.
+   * In contrast to {@link #iterator()}, a return value of {@code null}
+   * <b>does not</b> imply that no documents match the filter!
+   * The default implementation does not provide random access, so you
+   * only need to implement this method if your DocIdSet can
+   * guarantee random access to every docid in O(1) time without
+   * external disk access (as {@link Bits} interface cannot throw
+   * {@link IOException}). This is generally true for bit sets
+   * like {@link org.apache.lucene.util.FixedBitSet}, which return
+   * itsself if they are used as {@code DocIdSet}.
+   */
+  public Bits bits() throws IOException {
+    return null;
+  }
+
   /**
    * This method is a hint for {@link CachingWrapperFilter}, if this <code>DocIdSet</code>
    * should be cached without copying it into a BitSet. The default is to return

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java?rev=1197110&r1=1197109&r2=1197110&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java Thu Nov  3 13:35:07 2011
@@ -73,7 +73,7 @@ public abstract class FieldCacheRangeFil
   
   /** This method is implemented for each data type */
   @Override
-  public abstract DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException;
+  public abstract DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException;
 
   /**
    * Creates a string range filter using {@link FieldCache#getTermsIndex}. This works with all
@@ -83,7 +83,7 @@ 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(AtomicReaderContext context) throws IOException {
+      public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) 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);
@@ -122,9 +122,7 @@ public abstract class FieldCacheRangeFil
         
         assert inclusiveLowerPoint > 0 && inclusiveUpperPoint > 0;
         
-        // for this DocIdSet, we can ignore deleted docs
-        // because deleted docs have an order of 0 (null entry in StringIndex)
-        return new FieldCacheDocIdSet(context.reader, true) {
+        return new FieldCacheDocIdSet(context.reader.maxDoc(), acceptDocs) {
           @Override
           final boolean matchDoc(int doc) {
             final int docOrd = fcsi.getOrd(doc);
@@ -152,7 +150,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(AtomicReaderContext context) throws IOException {
+      public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
         final byte inclusiveLowerPoint, inclusiveUpperPoint;
         if (lowerVal != null) {
           final byte i = lowerVal.byteValue();
@@ -175,8 +173,7 @@ public abstract class FieldCacheRangeFil
           return DocIdSet.EMPTY_DOCIDSET;
         
         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(context.reader, !(inclusiveLowerPoint <= 0 && inclusiveUpperPoint >= 0)) {
+        return new FieldCacheDocIdSet(context.reader.maxDoc(), acceptDocs) {
           @Override
           boolean matchDoc(int doc) {
             return values[doc] >= inclusiveLowerPoint && values[doc] <= inclusiveUpperPoint;
@@ -203,7 +200,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(AtomicReaderContext context) throws IOException {
+      public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
         final short inclusiveLowerPoint, inclusiveUpperPoint;
         if (lowerVal != null) {
           short i = lowerVal.shortValue();
@@ -226,8 +223,7 @@ public abstract class FieldCacheRangeFil
           return DocIdSet.EMPTY_DOCIDSET;
         
         final short[] values = FieldCache.DEFAULT.getShorts(context.reader, field, (FieldCache.ShortParser) parser);
-        // ignore deleted docs if range doesn't contain 0
-        return new FieldCacheDocIdSet(context.reader, !(inclusiveLowerPoint <= 0 && inclusiveUpperPoint >= 0)) {
+        return new FieldCacheDocIdSet(context.reader.maxDoc(), acceptDocs) {
           @Override
           boolean matchDoc(int doc) {
             return values[doc] >= inclusiveLowerPoint && values[doc] <= inclusiveUpperPoint;
@@ -254,7 +250,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(AtomicReaderContext context) throws IOException {
+      public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
         final int inclusiveLowerPoint, inclusiveUpperPoint;
         if (lowerVal != null) {
           int i = lowerVal.intValue();
@@ -277,8 +273,7 @@ public abstract class FieldCacheRangeFil
           return DocIdSet.EMPTY_DOCIDSET;
         
         final int[] values = FieldCache.DEFAULT.getInts(context.reader, field, (FieldCache.IntParser) parser);
-        // ignore deleted docs if range doesn't contain 0
-        return new FieldCacheDocIdSet(context.reader, !(inclusiveLowerPoint <= 0 && inclusiveUpperPoint >= 0)) {
+        return new FieldCacheDocIdSet(context.reader.maxDoc(), acceptDocs) {
           @Override
           boolean matchDoc(int doc) {
             return values[doc] >= inclusiveLowerPoint && values[doc] <= inclusiveUpperPoint;
@@ -305,7 +300,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(AtomicReaderContext context) throws IOException {
+      public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
         final long inclusiveLowerPoint, inclusiveUpperPoint;
         if (lowerVal != null) {
           long i = lowerVal.longValue();
@@ -328,8 +323,7 @@ public abstract class FieldCacheRangeFil
           return DocIdSet.EMPTY_DOCIDSET;
         
         final long[] values = FieldCache.DEFAULT.getLongs(context.reader, field, (FieldCache.LongParser) parser);
-        // ignore deleted docs if range doesn't contain 0
-        return new FieldCacheDocIdSet(context.reader, !(inclusiveLowerPoint <= 0L && inclusiveUpperPoint >= 0L)) {
+        return new FieldCacheDocIdSet(context.reader.maxDoc(), acceptDocs) {
           @Override
           boolean matchDoc(int doc) {
             return values[doc] >= inclusiveLowerPoint && values[doc] <= inclusiveUpperPoint;
@@ -356,7 +350,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(AtomicReaderContext context) throws IOException {
+      public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) 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;
@@ -383,8 +377,7 @@ public abstract class FieldCacheRangeFil
           return DocIdSet.EMPTY_DOCIDSET;
         
         final float[] values = FieldCache.DEFAULT.getFloats(context.reader, field, (FieldCache.FloatParser) parser);
-        // ignore deleted docs if range doesn't contain 0
-        return new FieldCacheDocIdSet(context.reader, !(inclusiveLowerPoint <= 0.0f && inclusiveUpperPoint >= 0.0f)) {
+        return new FieldCacheDocIdSet(context.reader.maxDoc(), acceptDocs) {
           @Override
           boolean matchDoc(int doc) {
             return values[doc] >= inclusiveLowerPoint && values[doc] <= inclusiveUpperPoint;
@@ -411,7 +404,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(AtomicReaderContext context) throws IOException {
+      public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) 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;
@@ -439,7 +432,7 @@ public abstract class FieldCacheRangeFil
         
         final double[] values = FieldCache.DEFAULT.getDoubles(context.reader, field, (FieldCache.DoubleParser) parser);
         // ignore deleted docs if range doesn't contain 0
-        return new FieldCacheDocIdSet(context.reader, !(inclusiveLowerPoint <= 0.0 && inclusiveUpperPoint >= 0.0)) {
+        return new FieldCacheDocIdSet(context.reader.maxDoc(), acceptDocs) {
           @Override
           boolean matchDoc(int doc) {
             return values[doc] >= inclusiveLowerPoint && values[doc] <= inclusiveUpperPoint;
@@ -506,12 +499,12 @@ public abstract class FieldCacheRangeFil
   public FieldCache.Parser getParser() { return parser; }
   
   static abstract class FieldCacheDocIdSet extends DocIdSet {
-    private final IndexReader reader;
-    private final boolean canIgnoreDeletedDocs;
+    private final int maxDoc;
+    private final Bits acceptDocs;
 
-    FieldCacheDocIdSet(IndexReader reader, boolean canIgnoreDeletedDocs) {
-      this.reader = reader;
-      this.canIgnoreDeletedDocs = canIgnoreDeletedDocs;
+    FieldCacheDocIdSet(int maxDoc, Bits acceptDocs) {
+      this.maxDoc = maxDoc;
+      this.acceptDocs = acceptDocs;
     }
 
     /**
@@ -530,11 +523,29 @@ public abstract class FieldCacheRangeFil
     }
 
     @Override
-    public DocIdSetIterator iterator() throws IOException {
+    public Bits bits() {
+      return (acceptDocs == null) ? new Bits() {
+        public boolean get(int docid) {
+          return FieldCacheDocIdSet.this.matchDoc(docid);
+        }
+
+        public int length() {
+          return FieldCacheDocIdSet.this.maxDoc;
+        }
+      } : new Bits() {
+        public boolean get(int docid) {
+          return acceptDocs.get(docid) && FieldCacheDocIdSet.this.matchDoc(docid);
+        }
 
-      final Bits liveDocs = canIgnoreDeletedDocs ? null : reader.getLiveDocs();
+        public int length() {
+          return FieldCacheDocIdSet.this.maxDoc;
+        }
+      };
+    }
 
-      if (liveDocs == null) {
+    @Override
+    public DocIdSetIterator iterator() throws IOException {
+      if (acceptDocs == null) {
         // Specialization optimization disregard deletions
         return new DocIdSetIterator() {
           private int doc = -1;
@@ -569,12 +580,10 @@ public abstract class FieldCacheRangeFil
           }
         };
       } else {
-        // Must consult deletions
-
-        final int maxDoc = reader.maxDoc();
+        // Must consult acceptDocs
 
         // a DocIdSetIterator generating docIds by
-        // incrementing a variable & checking liveDocs -
+        // incrementing a variable & checking acceptDocs -
         return new DocIdSetIterator() {
           private int doc = -1;
           @Override
@@ -589,14 +598,14 @@ public abstract class FieldCacheRangeFil
               if (doc >= maxDoc) {
                 return doc = NO_MORE_DOCS;
               }
-            } while (!liveDocs.get(doc) || !matchDoc(doc));
+            } while (!acceptDocs.get(doc) || !matchDoc(doc));
             return doc;
           }
         
           @Override
           public int advance(int target) {
             for(doc=target;doc<maxDoc;doc++) {
-              if (liveDocs.get(doc) && matchDoc(doc)) {
+              if (acceptDocs.get(doc) && matchDoc(doc)) {
                 return doc;
               }
             }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java?rev=1197110&r1=1197109&r2=1197110&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java Thu Nov  3 13:35:07 2011
@@ -23,6 +23,7 @@ import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 
 /**
@@ -116,68 +117,22 @@ public class FieldCacheTermsFilter exten
   }
 
   @Override
-  public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
-    return new FieldCacheTermsFilterDocIdSet(getFieldCache().getTermsIndex(context.reader, field));
-  }
-
-  protected class FieldCacheTermsFilterDocIdSet extends DocIdSet {
-    private FieldCache.DocTermsIndex fcsi;
-
-    private FixedBitSet bits;
-
-    public FieldCacheTermsFilterDocIdSet(FieldCache.DocTermsIndex fcsi) {
-      this.fcsi = fcsi;
-      bits = new FixedBitSet(this.fcsi.numOrd());
-      final BytesRef spare = new BytesRef();
-      for (int i=0;i<terms.length;i++) {
-        int termNumber = this.fcsi.binarySearchLookup(terms[i], spare);
-        if (termNumber > 0) {
-          bits.set(termNumber);
-        }
+  public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
+    final FieldCache.DocTermsIndex fcsi = getFieldCache().getTermsIndex(context.reader, field);
+    final FixedBitSet bits = new FixedBitSet(fcsi.numOrd());
+    final BytesRef spare = new BytesRef();
+    for (int i=0;i<terms.length;i++) {
+      int termNumber = fcsi.binarySearchLookup(terms[i], spare);
+      if (termNumber > 0) {
+        bits.set(termNumber);
       }
     }
-
-    @Override
-    public DocIdSetIterator iterator() {
-      return new FieldCacheTermsFilterDocIdSetIterator();
-    }
-
-    /** This DocIdSet implementation is cacheable. */
-    @Override
-    public boolean isCacheable() {
-      return true;
-    }
-
-    protected class FieldCacheTermsFilterDocIdSetIterator extends DocIdSetIterator {
-      private int doc = -1;
-
-      @Override
-      public int docID() {
-        return doc;
-      }
-
+    final int maxDoc = context.reader.maxDoc();
+    return new FieldCacheRangeFilter.FieldCacheDocIdSet(maxDoc, acceptDocs) {
       @Override
-      public int nextDoc() {
-        try {
-          while (!bits.get(fcsi.getOrd(++doc))) {}
-        } catch (ArrayIndexOutOfBoundsException e) {
-          doc = NO_MORE_DOCS;
-        }
-        return doc;
+      boolean matchDoc(int doc) {
+        return bits.get(fcsi.getOrd(doc));
       }
-
-      @Override
-      public int advance(int target) {
-        try {
-          doc = target;
-          while (!bits.get(fcsi.getOrd(doc))) {
-            doc++;
-          }
-        } catch (ArrayIndexOutOfBoundsException e) {
-          doc = NO_MORE_DOCS;
-        }
-        return doc;
-      }
-    }
+    };
   }
 }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/Filter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/Filter.java?rev=1197110&r1=1197109&r2=1197110&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/Filter.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/Filter.java Thu Nov  3 13:35:07 2011
@@ -21,7 +21,7 @@ import java.io.IOException;
 
 import org.apache.lucene.index.IndexReader; // javadocs
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
-import org.apache.lucene.util.DocIdBitSet;
+import org.apache.lucene.util.Bits;
 
 /** 
  *  Abstract base class for restricting which documents may
@@ -44,14 +44,16 @@ public abstract class Filter {
    *         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#getLiveDocs()}
+   *         {@link IndexReader#fields()}
    *         on the context's reader, for example.
+   *
+   * @param acceptDocs
+   *          Bits that represent the allowable docs to match (typically deleted docs
+   *          but possibly filtering other documents)
    *          
    * @return a DocIdSet that provides the documents which should be permitted or
    *         prohibited in search results. <b>NOTE:</b> null can be returned if
    *         no documents will be accepted by this Filter.
-   * 
-   * @see DocIdBitSet
    */
-  public abstract DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException;
+  public abstract DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException;
 }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/FilteredDocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/FilteredDocIdSet.java?rev=1197110&r1=1197109&r2=1197110&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/FilteredDocIdSet.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/FilteredDocIdSet.java Thu Nov  3 13:35:07 2011
@@ -18,6 +18,7 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
+import org.apache.lucene.util.Bits;
 
 /**
  * Abstract decorator class for a DocIdSet implementation
@@ -54,14 +55,28 @@ public abstract class FilteredDocIdSet e
   public boolean isCacheable() {
     return _innerSet.isCacheable();
   }
+  
+  @Override
+  public Bits bits() throws IOException {
+    final Bits bits = _innerSet.bits();
+    return (bits == null) ? null : new Bits() {
+      public boolean get(int docid) {
+        return bits.get(docid) && FilteredDocIdSet.this.match(docid);
+      }
+
+      public int length() {
+        return bits.length();
+      }
+    };
+  }
 
   /**
    * Validation method to determine whether a docid should be in the result set.
    * @param docid docid to be tested
    * @return true if input docid should be in the result set, false otherwise.
    */
-  protected abstract boolean match(int docid) throws IOException;
-	
+  protected abstract boolean match(int docid);
+
   /**
    * Implementation of the contract to build a DocIdSetIterator.
    * @see DocIdSetIterator
@@ -71,7 +86,7 @@ public abstract class FilteredDocIdSet e
   public DocIdSetIterator iterator() throws IOException {
     return new FilteredDocIdSetIterator(_innerSet.iterator()) {
       @Override
-      protected boolean match(int docid) throws IOException {
+      protected boolean match(int docid) {
         return FilteredDocIdSet.this.match(docid);
       }
     };

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/FilteredDocIdSetIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/FilteredDocIdSetIterator.java?rev=1197110&r1=1197109&r2=1197110&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/FilteredDocIdSetIterator.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/FilteredDocIdSetIterator.java Thu Nov  3 13:35:07 2011
@@ -47,7 +47,7 @@ public abstract class FilteredDocIdSetIt
    * @return true if input docid should be in the result set, false otherwise.
    * @see #FilteredDocIdSetIterator(DocIdSetIterator)
    */
-  abstract protected boolean match(int doc) throws IOException;
+  protected abstract boolean match(int doc);
 	
   @Override
   public int docID() {

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/FilteredQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/FilteredQuery.java?rev=1197110&r1=1197109&r2=1197110&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/FilteredQuery.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/FilteredQuery.java Thu Nov  3 13:35:07 2011
@@ -55,6 +55,22 @@ extends Query {
     this.query = query;
     this.filter = filter;
   }
+  
+  /**
+   * Expert: decides if a filter should be executed as "random-access" or not.
+   * random-access means the filter "filters" in a similar way as deleted docs are filtered
+   * in lucene. This is faster when the filter accepts many documents.
+   * However, when the filter is very sparse, it can be faster to execute the query+filter
+   * as a conjunction in some cases.
+   * 
+   * The default implementation returns true if the first document accepted by the
+   * filter is < 100.
+   * 
+   * @lucene.internal
+   */
+  protected boolean useRandomAccess(Bits bits, int firstFilterDoc) {
+    return firstFilterDoc < 100;
+  }
 
   /**
    * Returns a Weight that applies the filter to the enclosed query's Weight.
@@ -66,6 +82,13 @@ extends Query {
     return new Weight() {
       
       @Override
+      public boolean scoresDocsOutOfOrder() {
+        // TODO: Support out-of-order scoring!
+        // For now we return false here, as we always get the scorer in order
+        return false;
+      }
+
+      @Override
       public float getValueForNormalization() throws IOException { 
         return weight.getValueForNormalization() * getBoost() * getBoost(); // boost sub-weight
       }
@@ -79,7 +102,7 @@ extends Query {
       public Explanation explain (AtomicReaderContext ir, int i) throws IOException {
         Explanation inner = weight.explain (ir, i);
         Filter f = FilteredQuery.this.filter;
-        DocIdSet docIdSet = f.getDocIdSet(ir);
+        DocIdSet docIdSet = f.getDocIdSet(ir, ir.reader.getLiveDocs());
         DocIdSetIterator docIdSetIterator = docIdSet == null ? DocIdSet.EMPTY_DOCIDSET.iterator() : docIdSet.iterator();
         if (docIdSetIterator == null) {
           docIdSetIterator = DocIdSet.EMPTY_DOCIDSET.iterator();
@@ -100,60 +123,108 @@ extends Query {
 
       // return a filtering scorer
       @Override
-      public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
-          boolean topScorer, Bits acceptDocs)
-          throws IOException {
-        // we will advance() the subscorer
-        final Scorer scorer = weight.scorer(context, true, false, acceptDocs);
-        if (scorer == null) {
+      public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer, Bits acceptDocs) throws IOException {
+        assert filter != null;
+
+        final DocIdSet filterDocIdSet = filter.getDocIdSet(context, acceptDocs);
+        if (filterDocIdSet == null) {
+          // this means the filter does not accept any documents.
           return null;
         }
-        DocIdSet docIdSet = filter.getDocIdSet(context);
-        if (docIdSet == null) {
+        
+        final DocIdSetIterator filterIter = filterDocIdSet.iterator();
+        if (filterIter == null) {
+          // this means the filter does not accept any documents.
           return null;
         }
-        final DocIdSetIterator docIdSetIterator = docIdSet.iterator();
-        if (docIdSetIterator == null) {
+
+        final int firstFilterDoc = filterIter.nextDoc();
+        if (firstFilterDoc == DocIdSetIterator.NO_MORE_DOCS) {
           return null;
         }
+        
+        final Bits filterAcceptDocs = filterDocIdSet.bits();
+        final boolean useRandomAccess = (filterAcceptDocs != null && FilteredQuery.this.useRandomAccess(filterAcceptDocs, firstFilterDoc));
+
+        if (useRandomAccess) {
+          // if we are using random access, we return the inner scorer, just with other acceptDocs
+          // TODO, replace this by when BooleanWeight is fixed to be consistent with its scorer implementations:
+          // return weight.scorer(context, scoreDocsInOrder, topScorer, filterAcceptDocs);
+          return weight.scorer(context, true, topScorer, filterAcceptDocs);
+        } else {
+          assert firstFilterDoc > -1;
+          // we are gonna advance() this scorer, so we set inorder=true/toplevel=false
+          // we pass null as acceptDocs, as our filter has already respected acceptDocs, no need to do twice
+          final Scorer scorer = weight.scorer(context, true, false, null);
+          return (scorer == null) ? null : new Scorer(this) {
+            private int scorerDoc = -1, filterDoc = firstFilterDoc;
+            
+            // optimization: we are topScorer and collect directly using short-circuited algo
+            @Override
+            public void score(Collector collector) throws IOException {
+              int filterDoc = firstFilterDoc;
+              int scorerDoc = scorer.advance(filterDoc);
+              // the normalization trick already applies the boost of this query,
+              // so we can use the wrapped scorer directly:
+              collector.setScorer(scorer);
+              for (;;) {
+                if (scorerDoc == filterDoc) {
+                  // Check if scorer has exhausted, only before collecting.
+                  if (scorerDoc == DocIdSetIterator.NO_MORE_DOCS) {
+                    break;
+                  }
+                  collector.collect(scorerDoc);
+                  filterDoc = filterIter.nextDoc();
+                  scorerDoc = scorer.advance(filterDoc);
+                } else if (scorerDoc > filterDoc) {
+                  filterDoc = filterIter.advance(scorerDoc);
+                } else {
+                  scorerDoc = scorer.advance(filterDoc);
+                }
+              }
+            }
+            
+            private int advanceToNextCommonDoc() throws IOException {
+              for (;;) {
+                if (scorerDoc < filterDoc) {
+                  scorerDoc = scorer.advance(filterDoc);
+                } else if (scorerDoc == filterDoc) {
+                  return scorerDoc;
+                } else {
+                  filterDoc = filterIter.advance(scorerDoc);
+                }
+              }
+            }
 
-        return new Scorer(this) {
-
-          private int doc = -1;
-          
-          private int advanceToCommon(int scorerDoc, int disiDoc) throws IOException {
-            while (scorerDoc != disiDoc) {
-              if (scorerDoc < disiDoc) {
-                scorerDoc = scorer.advance(disiDoc);
-              } else {
-                disiDoc = docIdSetIterator.advance(scorerDoc);
+            @Override
+            public int nextDoc() throws IOException {
+              // don't go to next doc on first call
+              // (because filterIter is already on first doc):
+              if (scorerDoc != -1) {
+                filterDoc = filterIter.nextDoc();
               }
+              return advanceToNextCommonDoc();
+            }
+            
+            @Override
+            public int advance(int target) throws IOException {
+              if (target > filterDoc) {
+                filterDoc = filterIter.advance(target);
+              }
+              return advanceToNextCommonDoc();
             }
-            return scorerDoc;
-          }
 
-          @Override
-          public int nextDoc() throws IOException {
-            int scorerDoc, disiDoc;
-            return doc = (disiDoc = docIdSetIterator.nextDoc()) != NO_MORE_DOCS
-                && (scorerDoc = scorer.nextDoc()) != NO_MORE_DOCS
-                && advanceToCommon(scorerDoc, disiDoc) != NO_MORE_DOCS ? scorer.docID() : NO_MORE_DOCS;
-          }
-          
-          @Override
-          public int docID() { return doc; }
-          
-          @Override
-          public int advance(int target) throws IOException {
-            int disiDoc, scorerDoc;
-            return doc = (disiDoc = docIdSetIterator.advance(target)) != NO_MORE_DOCS
-                && (scorerDoc = scorer.advance(disiDoc)) != NO_MORE_DOCS 
-                && advanceToCommon(scorerDoc, disiDoc) != NO_MORE_DOCS ? scorer.docID() : NO_MORE_DOCS;
-          }
-
-          @Override
-          public float score() throws IOException { return scorer.score(); }
-        };
+            @Override
+            public int docID() {
+              return scorerDoc;
+            }
+            
+            @Override
+            public float score() throws IOException {
+              return scorer.score();
+            }
+          };
+        }
       }
     };
   }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1197110&r1=1197109&r2=1197110&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/IndexSearcher.java Thu Nov  3 13:35:07 2011
@@ -41,6 +41,7 @@ import org.apache.lucene.search.similari
 import org.apache.lucene.search.similarities.SimilarityProvider;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.NIOFSDirectory;    // javadoc
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.ReaderUtil;
 import org.apache.lucene.util.ThreadInterruptedException;
 
@@ -273,6 +274,11 @@ public class IndexSearcher implements Cl
       reader.close();
     }
   }
+  
+  /** @lucene.internal */
+  protected Query wrapFilter(Query query, Filter filter) {
+    return (filter == null) ? query : new FilteredQuery(query, filter);
+  }
 
   /** Finds the top <code>n</code>
    * hits for <code>query</code> where all results are after a previous 
@@ -285,7 +291,7 @@ public class IndexSearcher implements Cl
    * @throws BooleanQuery.TooManyClauses
    */
   public TopDocs searchAfter(ScoreDoc after, Query query, int n) throws IOException {
-    return searchAfter(after, query, null, n);
+    return search(createNormalizedWeight(query), after, n);
   }
   
   /** Finds the top <code>n</code>
@@ -299,7 +305,7 @@ public class IndexSearcher implements Cl
    * @throws BooleanQuery.TooManyClauses
    */
   public TopDocs searchAfter(ScoreDoc after, Query query, Filter filter, int n) throws IOException {
-    return search(createNormalizedWeight(query), filter, after, n);
+    return search(createNormalizedWeight(wrapFilter(query, filter)), after, n);
   }
   
   /** Finds the top <code>n</code>
@@ -320,7 +326,7 @@ public class IndexSearcher implements Cl
    */
   public TopDocs search(Query query, Filter filter, int n)
     throws IOException {
-    return search(createNormalizedWeight(query), filter, null, n);
+    return search(createNormalizedWeight(wrapFilter(query, filter)), null, n);
   }
 
   /** Lower-level search API.
@@ -341,7 +347,7 @@ public class IndexSearcher implements Cl
    */
   public void search(Query query, Filter filter, Collector results)
     throws IOException {
-    search(leafContexts, createNormalizedWeight(query), filter, results);
+    search(leafContexts, createNormalizedWeight(wrapFilter(query, filter)), results);
   }
 
   /** Lower-level search API.
@@ -359,7 +365,7 @@ public class IndexSearcher implements Cl
   */
   public void search(Query query, Collector results)
     throws IOException {
-    search(leafContexts, createNormalizedWeight(query), null, results);
+    search(leafContexts, createNormalizedWeight(query), results);
   }
   
   /** Search implementation with arbitrary sorting.  Finds
@@ -375,7 +381,7 @@ public class IndexSearcher implements Cl
    */
   public TopFieldDocs search(Query query, Filter filter, int n,
                              Sort sort) throws IOException {
-    return search(createNormalizedWeight(query), filter, n, sort);
+    return search(createNormalizedWeight(wrapFilter(query, filter)), n, sort);
   }
 
   /**
@@ -388,7 +394,7 @@ public class IndexSearcher implements Cl
    */
   public TopFieldDocs search(Query query, int n,
                              Sort sort) throws IOException {
-    return search(createNormalizedWeight(query), null, n, sort);
+    return search(createNormalizedWeight(query), n, sort);
   }
 
   /** Expert: Low-level search implementation.  Finds the top <code>n</code>
@@ -398,9 +404,9 @@ public class IndexSearcher implements Cl
    * {@link IndexSearcher#search(Query,Filter,int)} instead.
    * @throws BooleanQuery.TooManyClauses
    */
-  protected TopDocs search(Weight weight, Filter filter, ScoreDoc after, int nDocs) throws IOException {
+  protected TopDocs search(Weight weight, ScoreDoc after, int nDocs) throws IOException {
     if (executor == null) {
-      return search(leafContexts, weight, filter, after, nDocs);
+      return search(leafContexts, weight, after, nDocs);
     } else {
       final HitQueue hq = new HitQueue(nDocs, false);
       final Lock lock = new ReentrantLock();
@@ -408,7 +414,7 @@ public class IndexSearcher implements Cl
     
       for (int i = 0; i < leafSlices.length; i++) { // search each sub
         runner.submit(
-                      new SearcherCallableNoSort(lock, this, leafSlices[i], weight, filter, after, nDocs, hq));
+                      new SearcherCallableNoSort(lock, this, leafSlices[i], weight, after, nDocs, hq));
       }
 
       int totalHits = 0;
@@ -429,13 +435,13 @@ public class IndexSearcher implements Cl
   }
 
   /** 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.
+   * hits for <code>query</code>.
    *
    * <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, ScoreDoc after, int nDocs) throws IOException {
+  protected TopDocs search(AtomicReaderContext[] leaves, Weight weight, ScoreDoc after, int nDocs) throws IOException {
     // single thread
     int limit = reader.maxDoc();
     if (limit == 0) {
@@ -443,37 +449,36 @@ public class IndexSearcher implements Cl
     }
     nDocs = Math.min(nDocs, limit);
     TopScoreDocCollector collector = TopScoreDocCollector.create(nDocs, after, !weight.scoresDocsOutOfOrder());
-    search(leaves, weight, filter, collector);
+    search(leaves, weight, 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>.
+   * the top <code>n</code> hits for <code>query</code> 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,
+  protected TopFieldDocs search(Weight weight,
       final int nDocs, Sort sort) throws IOException {
-    return search(weight, filter, nDocs, sort, true);
+    return search(weight, nDocs, sort, true);
   }
 
   /**
-   * Just like {@link #search(Weight, Filter, int, Sort)}, but you choose
+   * Just like {@link #search(Weight, 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(IndexReader.AtomicReaderContext[], Weight, Filter,
+   * then pass that to {@link #search(IndexReader.AtomicReaderContext[], Weight,
    * Collector)}.</p>
    */
-  protected TopFieldDocs search(Weight weight, Filter filter, int nDocs,
+  protected TopFieldDocs search(Weight weight, int nDocs,
                                 Sort sort, boolean fillFields)
       throws IOException {
 
@@ -481,7 +486,7 @@ public class IndexSearcher implements Cl
     
     if (executor == null) {
       // use all leaves here!
-      return search (leafContexts, weight, filter, nDocs, sort, fillFields);
+      return search (leafContexts, weight, nDocs, sort, fillFields);
     } else {
       final TopFieldCollector topCollector = TopFieldCollector.create(sort, nDocs,
                                                                       fillFields,
@@ -493,7 +498,7 @@ public class IndexSearcher implements Cl
       final ExecutionHelper<TopFieldDocs> runner = new ExecutionHelper<TopFieldDocs>(executor);
       for (int i = 0; i < leafSlices.length; i++) { // search each leaf slice
         runner.submit(
-                      new SearcherCallableWithSort(lock, this, leafSlices[i], weight, filter, nDocs, topCollector, sort));
+                      new SearcherCallableWithSort(lock, this, leafSlices[i], weight, nDocs, topCollector, sort));
       }
       int totalHits = 0;
       float maxScore = Float.NEGATIVE_INFINITY;
@@ -512,17 +517,17 @@ public class IndexSearcher implements Cl
   
   
   /**
-   * Just like {@link #search(Weight, Filter, int, Sort)}, but you choose
+   * Just like {@link #search(Weight, 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(IndexReader.AtomicReaderContext[], Weight, Filter,
+   * then pass that to {@link #search(IndexReader.AtomicReaderContext[], Weight, 
    * Collector)}.</p>
    */
-  protected TopFieldDocs search(AtomicReaderContext[] leaves, Weight weight, Filter filter, int nDocs,
+  protected TopFieldDocs search(AtomicReaderContext[] leaves, Weight weight, int nDocs,
       Sort sort, boolean fillFields) throws IOException {
     // single thread
     int limit = reader.maxDoc();
@@ -533,7 +538,7 @@ public class IndexSearcher implements Cl
 
     TopFieldCollector collector = TopFieldCollector.create(sort, nDocs,
                                                            fillFields, fieldSortDoTrackScores, fieldSortDoMaxScore, !weight.scoresDocsOutOfOrder());
-    search(leaves, weight, filter, collector);
+    search(leaves, weight, collector);
     return (TopFieldDocs) collector.topDocs();
   }
 
@@ -557,77 +562,21 @@ public class IndexSearcher implements Cl
    *          the searchers leaves to execute the searches on
    * @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(AtomicReaderContext[] leaves, Weight weight, Filter filter, Collector collector)
+  protected void search(AtomicReaderContext[] leaves, Weight weight, 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 < leaves.length; i++) { // search each subreader
-        collector.setNextReader(leaves[i]);
-        Scorer scorer = weight.scorer(leaves[i], !collector.acceptsDocsOutOfOrder(), true, leaves[i].reader.getLiveDocs());
-        if (scorer != null) {
-          scorer.score(collector);
-        }
-      }
-    } else {
-      for (int i = 0; i < leaves.length; i++) { // search each subreader
-        collector.setNextReader(leaves[i]);
-        searchWithFilter(leaves[i], weight, filter, collector);
-      }
-    }
-  }
-
-  private void searchWithFilter(AtomicReaderContext context, Weight weight,
-      final Filter filter, final Collector collector) throws IOException {
-
-    assert filter != null;
-    
-    // we are gonna advance() this scorer, so we set inorder=true/toplevel=false 
-    Scorer scorer = weight.scorer(context, true, false, context.reader.getLiveDocs());
-    if (scorer == null) {
-      return;
-    }
-
-    int docID = scorer.docID();
-    assert docID == -1 || docID == DocIdSetIterator.NO_MORE_DOCS;
-
-    // CHECKME: use ConjunctionScorer here?
-    DocIdSet filterDocIdSet = filter.getDocIdSet(context);
-    if (filterDocIdSet == null) {
-      // this means the filter does not accept any documents.
-      return;
-    }
-    
-    DocIdSetIterator filterIter = filterDocIdSet.iterator();
-    if (filterIter == null) {
-      // this means the filter does not accept any documents.
-      return;
-    }
-    int filterDoc = filterIter.nextDoc();
-    int scorerDoc = scorer.advance(filterDoc);
-    
-    collector.setScorer(scorer);
-    while (true) {
-      if (scorerDoc == filterDoc) {
-        // Check if scorer has exhausted, only before collecting.
-        if (scorerDoc == DocIdSetIterator.NO_MORE_DOCS) {
-          break;
-        }
-        collector.collect(scorerDoc);
-        filterDoc = filterIter.nextDoc();
-        scorerDoc = scorer.advance(filterDoc);
-      } else if (scorerDoc > filterDoc) {
-        filterDoc = filterIter.advance(scorerDoc);
-      } else {
-        scorerDoc = scorer.advance(filterDoc);
+    for (int i = 0; i < leaves.length; i++) { // search each subreader
+      collector.setNextReader(leaves[i]);
+      Scorer scorer = weight.scorer(leaves[i], !collector.acceptsDocsOutOfOrder(), true, leaves[i].reader.getLiveDocs());
+      if (scorer != null) {
+        scorer.score(collector);
       }
     }
   }
@@ -729,18 +678,16 @@ public class IndexSearcher implements Cl
     private final Lock lock;
     private final IndexSearcher searcher;
     private final Weight weight;
-    private final Filter filter;
     private final ScoreDoc after;
     private final int nDocs;
     private final HitQueue hq;
     private final LeafSlice slice;
 
     public SearcherCallableNoSort(Lock lock, IndexSearcher searcher, LeafSlice slice,  Weight weight,
-        Filter filter, ScoreDoc after, int nDocs, HitQueue hq) {
+        ScoreDoc after, int nDocs, HitQueue hq) {
       this.lock = lock;
       this.searcher = searcher;
       this.weight = weight;
-      this.filter = filter;
       this.after = after;
       this.nDocs = nDocs;
       this.hq = hq;
@@ -748,7 +695,7 @@ public class IndexSearcher implements Cl
     }
 
     public TopDocs call() throws IOException {
-      final TopDocs docs = searcher.search (slice.leaves, weight, filter, after, nDocs);
+      final TopDocs docs = searcher.search (slice.leaves, weight, after, nDocs);
       final ScoreDoc[] scoreDocs = docs.scoreDocs;
       //it would be so nice if we had a thread-safe insert 
       lock.lock();
@@ -775,18 +722,16 @@ public class IndexSearcher implements Cl
     private final Lock lock;
     private final IndexSearcher searcher;
     private final Weight weight;
-    private final Filter filter;
     private final int nDocs;
     private final TopFieldCollector hq;
     private final Sort sort;
     private final LeafSlice slice;
 
     public SearcherCallableWithSort(Lock lock, IndexSearcher searcher, LeafSlice slice, Weight weight,
-        Filter filter, int nDocs, TopFieldCollector hq, Sort sort) {
+        int nDocs, TopFieldCollector hq, Sort sort) {
       this.lock = lock;
       this.searcher = searcher;
       this.weight = weight;
-      this.filter = filter;
       this.nDocs = nDocs;
       this.hq = hq;
       this.sort = sort;
@@ -831,7 +776,7 @@ public class IndexSearcher implements Cl
 
     public TopFieldDocs call() throws IOException {
       assert slice.leaves.length == 1;
-      final TopFieldDocs docs = searcher.search (slice.leaves, weight, filter, nDocs, sort, true);
+      final TopFieldDocs docs = searcher.search (slice.leaves, weight, nDocs, sort, true);
       lock.lock();
       try {
         final int base = slice.leaves[0].docBase;

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java?rev=1197110&r1=1197109&r2=1197110&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java Thu Nov  3 13:35:07 2011
@@ -105,7 +105,7 @@ public class MultiTermQueryWrapperFilter
    * results.
    */
   @Override
-  public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
+  public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
     final IndexReader reader = context.reader;
     final Fields fields = reader.fields();
     if (fields == null) {
@@ -125,13 +125,12 @@ public class MultiTermQueryWrapperFilter
       // fill into a FixedBitSet
       final FixedBitSet bitSet = new FixedBitSet(context.reader.maxDoc());
       int termCount = 0;
-      final Bits liveDocs = reader.getLiveDocs();
       DocsEnum docsEnum = null;
       do {
         termCount++;
         // System.out.println("  iter termCount=" + termCount + " term=" +
         // enumerator.term().toBytesString());
-        docsEnum = termsEnum.docs(liveDocs, docsEnum);
+        docsEnum = termsEnum.docs(acceptDocs, docsEnum);
         final DocsEnum.BulkReadResult result = docsEnum.getBulkResult();
         while (true) {
           final int count = docsEnum.read();