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

svn commit: r1188624 [1/3] - in /lucene/dev/trunk: lucene/ lucene/contrib/misc/src/java/org/apache/lucene/index/ lucene/contrib/sandbox/src/java/org/apache/lucene/sandbox/queries/ lucene/contrib/spatial/src/java/org/apache/lucene/spatial/geohash/ lucen...

Author: uschindler
Date: Tue Oct 25 12:11:15 2011
New Revision: 1188624

URL: http://svn.apache.org/viewvc?rev=1188624&view=rev
Log:
LUCENE-1536: Filters can now be applied down-low, if their DocIdSet implements a new bits() method, returning all documents in a random access way

Added:
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/BitsFilteredDocIdSet.java   (with props)
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/index/PKIndexSplitter.java
    lucene/dev/trunk/lucene/contrib/sandbox/src/java/org/apache/lucene/sandbox/queries/DuplicateFilter.java
    lucene/dev/trunk/lucene/contrib/spatial/src/java/org/apache/lucene/spatial/geohash/GeoHashDistanceFilter.java
    lucene/dev/trunk/lucene/contrib/spatial/src/java/org/apache/lucene/spatial/tier/CartesianShapeFilter.java
    lucene/dev/trunk/lucene/contrib/spatial/src/java/org/apache/lucene/spatial/tier/LatLongDistanceFilter.java
    lucene/dev/trunk/lucene/contrib/spatial/src/test/org/apache/lucene/spatial/tier/TestDistance.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/CachingSpanFilter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/CachingWrapperFilter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/DocIdSet.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/Filter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FilteredDocIdSet.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FilteredDocIdSetIterator.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FilteredQuery.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/IndexSearcher.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/QueryWrapperFilter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/SpanFilter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/SpanFilterResult.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/SpanQueryFilter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/DocIdBitSet.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/FixedBitSet.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/OpenBitSet.java
    lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/search/AssertingIndexSearcher.java
    lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/search/CachingWrapperFilterHelper.java
    lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/FieldCacheRewriteMethod.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/JustCompileSearch.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/MockFilter.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/SingleDocTestFilter.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestCachingSpanFilter.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestCachingWrapperFilter.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestConstantScoreQuery.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestDocIdSet.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestFilteredQuery.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestFilteredSearch.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestNumericRangeQuery32.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestNumericRangeQuery64.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestScorerPerf.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestSort.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestSpanQueryFilter.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestTopDocsMerge.java
    lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java
    lucene/dev/trunk/modules/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java
    lucene/dev/trunk/modules/join/src/java/org/apache/lucene/search/join/BlockJoinQuery.java
    lucene/dev/trunk/modules/queries/src/java/org/apache/lucene/queries/BooleanFilter.java
    lucene/dev/trunk/modules/queries/src/java/org/apache/lucene/queries/ChainedFilter.java
    lucene/dev/trunk/modules/queries/src/java/org/apache/lucene/queries/TermsFilter.java
    lucene/dev/trunk/modules/queries/src/test/org/apache/lucene/queries/BooleanFilterTest.java
    lucene/dev/trunk/modules/queries/src/test/org/apache/lucene/queries/TermsFilterTest.java
    lucene/dev/trunk/modules/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/NumericRangeFilterBuilder.java
    lucene/dev/trunk/modules/queryparser/src/test/org/apache/lucene/queryparser/xml/builders/TestNumericRangeFilterBuilder.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/request/PerSegmentSingleValuedFaceting.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/BitDocSet.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/DocSet.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SolrFilter.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SortedIntDocSet.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/function/ValueSourceRangeFilter.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestDocSet.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestSort.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1188624&r1=1188623&r2=1188624&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Tue Oct 25 12:11:15 2011
@@ -189,6 +189,12 @@ Changes in backwards compatibility polic
   -1 to be consistent with this behavior across other index statistics.
   (Robert Muir)
 
+* LUCENE-1536: The abstract FilteredDocIdSet.match() method is no longer
+  allowed to throw IOException. This change was required to make it conform
+  to the Bits interface. This method should never do I/O for performance reasons.
+  (Mike McCandless, Uwe Schindler, Robert Muir, Chris Male, Yonik Seeley,
+  Jason Rutherglen, Paul Elschot)
+  
 Changes in Runtime Behavior
 
 * LUCENE-2846: omitNorms now behaves like omitTermFrequencyAndPositions, if you
@@ -338,6 +344,15 @@ API Changes
   case, and fixed places in Lucene to use seekExact when possible.
   (Mike McCandless)
 
+* LUCENE-1536: Filter.getDocIdSet() now takes an acceptDocs Bits interface (like
+  Scorer) limiting the documents that can appear in the returned DocIdSet.
+  Filters are now required to respect these acceptDocs, otherwise deleted documents
+  may get returned by searches. Most filters will pass these Bits down to DocsEnum,
+  but those, e.g. working on FieldCache, may need to use BitsFilteredDocIdSet.wrap()
+  to exclude them.
+  (Mike McCandless, Uwe Schindler, Robert Muir, Chris Male, Yonik Seeley,
+  Jason Rutherglen, Paul Elschot)
+
 New features
 
 * LUCENE-2604: Added RegexpQuery support to QueryParser. Regular expressions
@@ -558,6 +573,19 @@ New features
   resident and disk resident IndexDocValues are now exposed via the Source
   interface. ValuesEnum has been removed in favour of Source. (Simon Willnauer)
 
+* LUCENE-1536: Filters can now be applied down-low, if their DocIdSet implements
+  a new bits() method, returning all documents in a random access way. If the
+  DocIdSet is not too sparse, it will be passed as acceptDocs down to the Scorer
+  as replacement for IndexReader's live docs.
+  In addition, FilteredQuery backs now IndexSearcher's filtering search methods.
+  Using FilteredQuery you can chain Filters in a very performant way
+  [new FilteredQuery(new FilteredQuery(query, filter1), filter2)], which was not
+  possible with IndexSearcher's methods. FilteredQuery also allows to override
+  the heuristics used to decide if filtering should be done random access or
+  using a conjunction on DocIdSet's iterator().
+  (Mike McCandless, Uwe Schindler, Robert Muir, Chris Male, Yonik Seeley,
+  Jason Rutherglen, Paul Elschot)
+
 Optimizations
 
 * LUCENE-2588: Don't store unnecessary suffixes when writing the terms

Modified: lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/index/PKIndexSplitter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/index/PKIndexSplitter.java?rev=1188624&r1=1188623&r2=1188624&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/index/PKIndexSplitter.java (original)
+++ lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/index/PKIndexSplitter.java Tue Oct 25 12:11:15 2011
@@ -121,7 +121,8 @@ public class PKIndexSplitter {
       
       final int maxDoc = in.maxDoc();
       final FixedBitSet bits = new FixedBitSet(maxDoc);
-      final DocIdSet docs = preserveFilter.getDocIdSet((AtomicReaderContext) in.getTopReaderContext());
+      // ignore livedocs here, as we filter them later:
+      final DocIdSet docs = preserveFilter.getDocIdSet((AtomicReaderContext) in.getTopReaderContext(), null);
       if (docs != null) {
         final DocIdSetIterator it = docs.iterator();
         if (it != null) {

Modified: lucene/dev/trunk/lucene/contrib/sandbox/src/java/org/apache/lucene/sandbox/queries/DuplicateFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/sandbox/src/java/org/apache/lucene/sandbox/queries/DuplicateFilter.java?rev=1188624&r1=1188623&r2=1188624&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/sandbox/src/java/org/apache/lucene/sandbox/queries/DuplicateFilter.java (original)
+++ lucene/dev/trunk/lucene/contrib/sandbox/src/java/org/apache/lucene/sandbox/queries/DuplicateFilter.java Tue Oct 25 12:11:15 2011
@@ -70,17 +70,16 @@ public class DuplicateFilter extends Fil
   }
 
   @Override
-  public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
+  public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
     if (processingMode == ProcessingMode.PM_FAST_INVALIDATION) {
-      return fastBits(context.reader);
+      return fastBits(context.reader, acceptDocs);
     } else {
-      return correctBits(context.reader);
+      return correctBits(context.reader, acceptDocs);
     }
   }
 
-  private FixedBitSet correctBits(IndexReader reader) throws IOException {
+  private FixedBitSet correctBits(IndexReader reader, Bits acceptDocs) throws IOException {
     FixedBitSet bits = new FixedBitSet(reader.maxDoc()); //assume all are INvalid
-    final Bits liveDocs = MultiFields.getLiveDocs(reader);
     Terms terms = reader.fields().terms(fieldName);
 
     if (terms == null) {
@@ -94,7 +93,7 @@ public class DuplicateFilter extends Fil
       if (currTerm == null) {
         break;
       } else {
-        docs = termsEnum.docs(liveDocs, docs);
+        docs = termsEnum.docs(acceptDocs, docs);
         int doc = docs.nextDoc();
         if (doc != DocsEnum.NO_MORE_DOCS) {
           if (keepMode == KeepMode.KM_USE_FIRST_OCCURRENCE) {
@@ -116,10 +115,9 @@ public class DuplicateFilter extends Fil
     return bits;
   }
 
-  private FixedBitSet fastBits(IndexReader reader) throws IOException {
+  private FixedBitSet fastBits(IndexReader reader, Bits acceptDocs) throws IOException {
     FixedBitSet bits = new FixedBitSet(reader.maxDoc());
     bits.set(0, reader.maxDoc()); //assume all are valid
-    final Bits liveDocs = MultiFields.getLiveDocs(reader);
     Terms terms = reader.fields().terms(fieldName);
 
     if (terms == null) {
@@ -135,7 +133,7 @@ public class DuplicateFilter extends Fil
       } else {
         if (termsEnum.docFreq() > 1) {
           // unset potential duplicates
-          docs = termsEnum.docs(liveDocs, docs);
+          docs = termsEnum.docs(acceptDocs, docs);
           int doc = docs.nextDoc();
           if (doc != DocsEnum.NO_MORE_DOCS) {
             if (keepMode == KeepMode.KM_USE_FIRST_OCCURRENCE) {

Modified: lucene/dev/trunk/lucene/contrib/spatial/src/java/org/apache/lucene/spatial/geohash/GeoHashDistanceFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/spatial/src/java/org/apache/lucene/spatial/geohash/GeoHashDistanceFilter.java?rev=1188624&r1=1188623&r2=1188624&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/spatial/src/java/org/apache/lucene/spatial/geohash/GeoHashDistanceFilter.java (original)
+++ lucene/dev/trunk/lucene/contrib/spatial/src/java/org/apache/lucene/spatial/geohash/GeoHashDistanceFilter.java Tue Oct 25 12:11:15 2011
@@ -26,6 +26,7 @@ import org.apache.lucene.search.Filter;
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.FilteredDocIdSet;
 import org.apache.lucene.spatial.DistanceUtils;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.spatial.tier.DistanceFilter;
 
@@ -57,7 +58,7 @@ public class GeoHashDistanceFilter exten
   }
 
   @Override
-  public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
+  public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
 
     final DocTerms geoHashValues = FieldCache.DEFAULT.getTerms(context.reader, geoHashField);
     final BytesRef br = new BytesRef();
@@ -65,7 +66,7 @@ public class GeoHashDistanceFilter exten
     final int docBase = nextDocBase;
     nextDocBase += context.reader.maxDoc();
 
-    return new FilteredDocIdSet(startingFilter.getDocIdSet(context)) {
+    return new FilteredDocIdSet(startingFilter.getDocIdSet(context, acceptDocs)) {
       @Override
       public boolean match(int doc) {
 

Modified: lucene/dev/trunk/lucene/contrib/spatial/src/java/org/apache/lucene/spatial/tier/CartesianShapeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/spatial/src/java/org/apache/lucene/spatial/tier/CartesianShapeFilter.java?rev=1188624&r1=1188623&r2=1188624&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/spatial/src/java/org/apache/lucene/spatial/tier/CartesianShapeFilter.java (original)
+++ lucene/dev/trunk/lucene/contrib/spatial/src/java/org/apache/lucene/spatial/tier/CartesianShapeFilter.java Tue Oct 25 12:11:15 2011
@@ -45,8 +45,7 @@ public class CartesianShapeFilter extend
   }
   
   @Override
-  public DocIdSet getDocIdSet(final AtomicReaderContext context) throws IOException {
-    final Bits liveDocs = context.reader.getLiveDocs();
+  public DocIdSet getDocIdSet(final AtomicReaderContext context, final Bits acceptDocs) throws IOException {
     final List<Double> area = shape.getArea();
     final int sz = area.size();
     
@@ -58,7 +57,7 @@ public class CartesianShapeFilter extend
       return new DocIdSet() {
         @Override
         public DocIdSetIterator iterator() throws IOException {
-          return context.reader.termDocsEnum(liveDocs, fieldName, bytesRef);
+          return context.reader.termDocsEnum(acceptDocs, fieldName, bytesRef);
         }
         
         @Override
@@ -71,7 +70,7 @@ public class CartesianShapeFilter extend
       for (int i =0; i< sz; i++) {
         double boxId = area.get(i).doubleValue();
         NumericUtils.longToPrefixCoded(NumericUtils.doubleToSortableLong(boxId), 0, bytesRef);
-        final DocsEnum docsEnum = context.reader.termDocsEnum(liveDocs, fieldName, bytesRef);
+        final DocsEnum docsEnum = context.reader.termDocsEnum(acceptDocs, fieldName, bytesRef);
         if (docsEnum == null) continue;
         // iterate through all documents
         // which have this boxId

Modified: lucene/dev/trunk/lucene/contrib/spatial/src/java/org/apache/lucene/spatial/tier/LatLongDistanceFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/spatial/src/java/org/apache/lucene/spatial/tier/LatLongDistanceFilter.java?rev=1188624&r1=1188623&r2=1188624&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/spatial/src/java/org/apache/lucene/spatial/tier/LatLongDistanceFilter.java (original)
+++ lucene/dev/trunk/lucene/contrib/spatial/src/java/org/apache/lucene/spatial/tier/LatLongDistanceFilter.java Tue Oct 25 12:11:15 2011
@@ -24,6 +24,7 @@ import org.apache.lucene.search.Filtered
 import org.apache.lucene.search.FieldCache;
 import org.apache.lucene.search.Filter;
 import org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.spatial.DistanceUtils;
 
 
@@ -60,7 +61,7 @@ public class LatLongDistanceFilter exten
   }
   
   @Override
-  public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
+  public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
 
     final double[] latIndex = FieldCache.DEFAULT.getDoubles(context.reader, latField);
     final double[] lngIndex = FieldCache.DEFAULT.getDoubles(context.reader, lngField);
@@ -68,7 +69,7 @@ public class LatLongDistanceFilter exten
     final int docBase = nextDocBase;
     nextDocBase += context.reader.maxDoc();
 
-    return new FilteredDocIdSet(startingFilter.getDocIdSet(context)) {
+    return new FilteredDocIdSet(startingFilter.getDocIdSet(context, acceptDocs)) {
       @Override
       protected boolean match(int doc) {
         double x = latIndex[doc];

Modified: lucene/dev/trunk/lucene/contrib/spatial/src/test/org/apache/lucene/spatial/tier/TestDistance.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/spatial/src/test/org/apache/lucene/spatial/tier/TestDistance.java?rev=1188624&r1=1188623&r2=1188624&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/spatial/src/test/org/apache/lucene/spatial/tier/TestDistance.java (original)
+++ lucene/dev/trunk/lucene/contrib/spatial/src/test/org/apache/lucene/spatial/tier/TestDistance.java Tue Oct 25 12:11:15 2011
@@ -105,7 +105,7 @@ public class TestDistance extends Lucene
 
     AtomicReaderContext[] leaves = ReaderUtil.leaves(r.getTopReaderContext());
     for (int i = 0; i < leaves.length; i++) {
-      f.getDocIdSet(leaves[i]);
+      f.getDocIdSet(leaves[i], leaves[i].reader.getLiveDocs());
     }
     r.close();
   }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java?rev=1188624&r1=1188623&r2=1188624&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java Tue Oct 25 12:11:15 2011
@@ -438,7 +438,7 @@ class BufferedDeletesStream {
     for (QueryAndLimit ent : queriesIter) {
       Query query = ent.query;
       int limit = ent.limit;
-      final DocIdSet docs = new QueryWrapperFilter(query).getDocIdSet(readerContext);
+      final DocIdSet docs = new QueryWrapperFilter(query).getDocIdSet(readerContext, readerContext.reader.getLiveDocs());
       if (docs != null) {
         final DocIdSetIterator it = docs.iterator();
         if (it != null) {
@@ -448,11 +448,8 @@ class BufferedDeletesStream {
               break;
 
             reader.deleteDocument(doc);
-            // TODO: we could/should change
-            // reader.deleteDocument to return boolean
-            // true if it did in fact delete, because here
-            // we could be deleting an already-deleted doc
-            // which makes this an upper bound:
+            // as we use getLiveDocs() to filter out already deleted documents,
+            // we only delete live documents, so the counting is right:
             delCount++;
           }
         }

Added: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/BitsFilteredDocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/BitsFilteredDocIdSet.java?rev=1188624&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/BitsFilteredDocIdSet.java (added)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/BitsFilteredDocIdSet.java Tue Oct 25 12:11:15 2011
@@ -0,0 +1,63 @@
+package org.apache.lucene.search;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.util.Bits;
+
+/**
+ * This implementation supplies a filtered DocIdSet, that excludes all
+ * docids which are not in a Bits instance. This is especially useful in
+ * {@link org.apache.lucene.search.Filter} to apply the {@code acceptDocs}
+ * passed to {@code getDocIdSet()} before returning the final DocIdSet.
+ *
+ * @see DocIdSet
+ * @see org.apache.lucene.search.Filter
+ */
+
+public final class BitsFilteredDocIdSet extends FilteredDocIdSet {
+
+  private final Bits acceptDocs;
+  
+  /**
+   * Convenience wrapper method: If {@code acceptDocs == null} it returns the original set without wrapping.
+   * @param set Underlying DocIdSet. If {@code null}, this method returns {@code null}
+   * @param acceptDocs Allowed docs, all docids not in this set will not be returned by this DocIdSet.
+   * If {@code null}, this method returns the original set without wrapping.
+   */
+  public static DocIdSet wrap(DocIdSet set, Bits acceptDocs) {
+    return (set == null || acceptDocs == null) ? set : new BitsFilteredDocIdSet(set, acceptDocs);
+  }
+  
+  /**
+   * Constructor.
+   * @param innerSet Underlying DocIdSet
+   * @param acceptDocs Allowed docs, all docids not in this set will not be returned by this DocIdSet
+   */
+  public BitsFilteredDocIdSet(DocIdSet innerSet, Bits acceptDocs) {
+    super(innerSet);
+    if (acceptDocs == null)
+      throw new NullPointerException("acceptDocs is null");
+    this.acceptDocs = acceptDocs;
+  }
+
+  @Override
+  protected boolean match(int docid) {
+    return acceptDocs.get(docid);
+  }
+
+}

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/CachingSpanFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/CachingSpanFilter.java?rev=1188624&r1=1188623&r2=1188624&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/CachingSpanFilter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/CachingSpanFilter.java Tue Oct 25 12:11:15 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
@@ -40,30 +43,59 @@ public class CachingSpanFilter extends S
    * @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 +103,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/trunk/lucene/src/java/org/apache/lucene/search/CachingWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/CachingWrapperFilter.java?rev=1188624&r1=1188623&r2=1188624&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/CachingWrapperFilter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/CachingWrapperFilter.java Tue Oct 25 12:11:15 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,29 @@ 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}.
+   * cache hit rate on reopened readers.
    * @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 +104,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/trunk/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java?rev=1188624&r1=1188623&r2=1188624&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java Tue Oct 25 12:11:15 2011
@@ -128,11 +128,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/trunk/lucene/src/java/org/apache/lucene/search/DocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/DocIdSet.java?rev=1188624&r1=1188623&r2=1188624&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/DocIdSet.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/DocIdSet.java Tue Oct 25 12:11:15 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/trunk/lucene/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java?rev=1188624&r1=1188623&r2=1188624&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java Tue Oct 25 12:11:15 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/trunk/lucene/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java?rev=1188624&r1=1188623&r2=1188624&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java Tue Oct 25 12:11:15 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/trunk/lucene/src/java/org/apache/lucene/search/Filter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/Filter.java?rev=1188624&r1=1188623&r2=1188624&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/Filter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/Filter.java Tue Oct 25 12:11:15 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/trunk/lucene/src/java/org/apache/lucene/search/FilteredDocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FilteredDocIdSet.java?rev=1188624&r1=1188623&r2=1188624&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FilteredDocIdSet.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FilteredDocIdSet.java Tue Oct 25 12:11:15 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/trunk/lucene/src/java/org/apache/lucene/search/FilteredDocIdSetIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FilteredDocIdSetIterator.java?rev=1188624&r1=1188623&r2=1188624&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FilteredDocIdSetIterator.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FilteredDocIdSetIterator.java Tue Oct 25 12:11:15 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/trunk/lucene/src/java/org/apache/lucene/search/FilteredQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FilteredQuery.java?rev=1188624&r1=1188623&r2=1188624&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FilteredQuery.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FilteredQuery.java Tue Oct 25 12:11:15 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/trunk/lucene/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1188624&r1=1188623&r2=1188624&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/IndexSearcher.java Tue Oct 25 12:11:15 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/trunk/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java?rev=1188624&r1=1188623&r2=1188624&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java Tue Oct 25 12:11:15 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();