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

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1188624 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Uwe Schindler 2011-10-25 12:11:15 +00:00
parent a91efbedd1
commit 94f98ab7f3
68 changed files with 925 additions and 667 deletions

View File

@ -189,6 +189,12 @@ Changes in backwards compatibility policy
-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

View File

@ -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) {

View File

@ -70,17 +70,16 @@ public class DuplicateFilter extends Filter {
}
@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 Filter {
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 Filter {
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 Filter {
} 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) {

View File

@ -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 extends DistanceFilter {
}
@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 extends DistanceFilter {
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) {

View File

@ -45,8 +45,7 @@ public class CartesianShapeFilter extends Filter {
}
@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 extends Filter {
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 extends Filter {
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

View File

@ -24,6 +24,7 @@ import org.apache.lucene.search.FilteredDocIdSet;
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 extends DistanceFilter {
}
@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 extends DistanceFilter {
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];

View File

@ -105,7 +105,7 @@ public class TestDistance extends LuceneTestCase {
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();
}

View File

@ -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++;
}
}

View File

@ -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);
}
}

View File

@ -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 SpanFilter {
* @param filter Filter to cache results of
*/
public CachingSpanFilter(SpanFilter filter) {
this(filter, CachingWrapperFilter.DeletesMode.RECACHE);
}
/**
* @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");
}
};
this.cache = new CachingWrapperFilter.FilterCache<SpanFilterResult>();
}
@Override
public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
SpanFilterResult result = getCachedResult(context);
return result != null ? result.getDocIdSet() : null;
public DocIdSet getDocIdSet(AtomicReaderContext context, final Bits acceptDocs) throws IOException {
final SpanFilterResult result = getCachedResult(context);
return BitsFilteredDocIdSet.wrap(result.getDocIdSet(), acceptDocs);
}
@Override
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 SpanFilter {
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+")";

View File

@ -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 extends Filter {
// 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 extends Filter {
// 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 extends Filter {
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

View File

@ -128,11 +128,11 @@ public class ConstantScoreQuery extends Query {
@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;
}

View File

@ -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

View File

@ -73,7 +73,7 @@ public abstract class FieldCacheRangeFilter<T> extends Filter {
/** 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 FieldCacheRangeFilter<T> extends Filter {
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 FieldCacheRangeFilter<T> extends Filter {
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 FieldCacheRangeFilter<T> extends Filter {
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 FieldCacheRangeFilter<T> extends Filter {
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 FieldCacheRangeFilter<T> extends Filter {
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 FieldCacheRangeFilter<T> extends Filter {
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 FieldCacheRangeFilter<T> extends Filter {
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 FieldCacheRangeFilter<T> extends Filter {
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 FieldCacheRangeFilter<T> extends Filter {
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 FieldCacheRangeFilter<T> extends Filter {
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 FieldCacheRangeFilter<T> extends Filter {
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 FieldCacheRangeFilter<T> extends Filter {
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 FieldCacheRangeFilter<T> extends Filter {
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 FieldCacheRangeFilter<T> extends Filter {
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 FieldCacheRangeFilter<T> extends Filter {
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;
}
/**
@ -529,12 +522,30 @@ public abstract class FieldCacheRangeFilter<T> extends Filter {
return true;
}
@Override
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);
}
public int length() {
return FieldCacheDocIdSet.this.maxDoc;
}
};
}
@Override
public DocIdSetIterator iterator() throws IOException {
final Bits liveDocs = canIgnoreDeletedDocs ? null : reader.getLiveDocs();
if (liveDocs == null) {
if (acceptDocs == null) {
// Specialization optimization disregard deletions
return new DocIdSetIterator() {
private int doc = -1;
@ -569,12 +580,10 @@ public abstract class FieldCacheRangeFilter<T> extends Filter {
}
};
} 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 FieldCacheRangeFilter<T> extends Filter {
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;
}
}

View File

@ -23,6 +23,7 @@ import org.apache.lucene.index.DocsEnum; // javadoc @link
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
/**
@ -116,68 +117,22 @@ public class FieldCacheTermsFilter extends Filter {
}
@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;
final int maxDoc = context.reader.maxDoc();
return new FieldCacheRangeFilter.FieldCacheDocIdSet(maxDoc, acceptDocs) {
@Override
public int docID() {
return doc;
boolean matchDoc(int doc) {
return bits.get(fcsi.getOrd(doc));
}
@Override
public int nextDoc() {
try {
while (!bits.get(fcsi.getOrd(++doc))) {}
} catch (ArrayIndexOutOfBoundsException e) {
doc = NO_MORE_DOCS;
}
return 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;
}
}
};
}
}

View File

@ -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;
}

View File

@ -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 extends DocIdSet {
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 extends DocIdSet {
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);
}
};

View File

@ -47,7 +47,7 @@ public abstract class FilteredDocIdSetIterator extends DocIdSetIterator {
* @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() {

View File

@ -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.
@ -65,6 +81,13 @@ extends Query {
final Weight weight = query.createWeight (searcher);
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) {
return null;
}
final DocIdSetIterator docIdSetIterator = docIdSet.iterator();
if (docIdSetIterator == null) {
final DocIdSetIterator filterIter = filterDocIdSet.iterator();
if (filterIter == null) {
// this means the filter does not accept any documents.
return null;
}
return new Scorer(this) {
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));
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);
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 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 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();
}
@Override
public float score() throws IOException { return scorer.score(); }
};
@Override
public int docID() {
return scorerDoc;
}
@Override
public float score() throws IOException {
return scorer.score();
}
};
}
}
};
}

View File

@ -41,6 +41,7 @@ import org.apache.lucene.search.similarities.DefaultSimilarityProvider;
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 Closeable {
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 Closeable {
* @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 Closeable {
* @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 Closeable {
*/
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 Closeable {
*/
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 Closeable {
*/
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 Closeable {
*/
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 Closeable {
*/
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 Closeable {
* {@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 Closeable {
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 Closeable {
}
/** 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 Closeable {
}
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 Closeable {
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 Closeable {
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 Closeable {
/**
* 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 Closeable {
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 Closeable {
* 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 Closeable {
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 Closeable {
}
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 Closeable {
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 Closeable {
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;

View File

@ -105,7 +105,7 @@ public class MultiTermQueryWrapperFilter<Q extends MultiTermQuery> extends Filte
* 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<Q extends MultiTermQuery> extends Filte
// 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();

View File

@ -20,6 +20,7 @@ package org.apache.lucene.search;
import java.io.IOException;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.util.Bits;
/**
* Constrains search results to only match those which also match a provided
@ -47,7 +48,7 @@ public class QueryWrapperFilter extends Filter {
}
@Override
public DocIdSet getDocIdSet(final AtomicReaderContext context) throws IOException {
public DocIdSet getDocIdSet(final AtomicReaderContext context, final Bits acceptDocs) throws IOException {
// get a private context that is used to rewrite, createWeight and score eventually
assert context.reader.getTopReaderContext().isAtomic;
final AtomicReaderContext privateContext = (AtomicReaderContext) context.reader.getTopReaderContext();
@ -55,7 +56,7 @@ public class QueryWrapperFilter extends Filter {
return new DocIdSet() {
@Override
public DocIdSetIterator iterator() throws IOException {
return weight.scorer(privateContext, true, false, privateContext.reader.getLiveDocs());
return weight.scorer(privateContext, true, false, acceptDocs);
}
@Override
public boolean isCacheable() { return false; }

View File

@ -16,6 +16,7 @@ package org.apache.lucene.search;
*/
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.util.Bits;
import java.io.IOException;
@ -34,5 +35,5 @@ public abstract class SpanFilter extends Filter{
* @return A {@link SpanFilterResult}
* @throws java.io.IOException if there was an issue accessing the necessary information
* */
public abstract SpanFilterResult bitSpans(AtomicReaderContext context) throws IOException;
public abstract SpanFilterResult bitSpans(AtomicReaderContext context, Bits acceptDocs) throws IOException;
}

View File

@ -16,7 +16,7 @@ package org.apache.lucene.search;
*/
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
@ -30,6 +30,9 @@ public class SpanFilterResult {
private DocIdSet docIdSet;
private List<PositionInfo> positions;//Spans spans;
public static final SpanFilterResult EMPTY_SPAN_FILTER_RESULT =
new SpanFilterResult(DocIdSet.EMPTY_DOCIDSET, Collections.<PositionInfo>emptyList());
/**
*
* @param docIdSet The DocIdSet for the Filter

View File

@ -19,6 +19,7 @@ package org.apache.lucene.search;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.search.spans.SpanQuery;
import org.apache.lucene.search.spans.Spans;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.FixedBitSet;
import java.io.IOException;
@ -52,16 +53,16 @@ public class SpanQueryFilter extends SpanFilter {
}
@Override
public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
SpanFilterResult result = bitSpans(context);
public final DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
SpanFilterResult result = bitSpans(context, acceptDocs);
return result.getDocIdSet();
}
@Override
public SpanFilterResult bitSpans(AtomicReaderContext context) throws IOException {
public SpanFilterResult bitSpans(AtomicReaderContext context, Bits acceptDocs) throws IOException {
final FixedBitSet bits = new FixedBitSet(context.reader.maxDoc());
Spans spans = query.getSpans(context, context.reader.getLiveDocs());
Spans spans = query.getSpans(context, acceptDocs);
List<SpanFilterResult.PositionInfo> tmp = new ArrayList<SpanFilterResult.PositionInfo>(20);
int currentDoc = -1;
SpanFilterResult.PositionInfo currentInfo = null;

View File

@ -24,8 +24,8 @@ import org.apache.lucene.search.DocIdSetIterator;
/** Simple DocIdSet and DocIdSetIterator backed by a BitSet */
public class DocIdBitSet extends DocIdSet {
private BitSet bitSet;
public class DocIdBitSet extends DocIdSet implements Bits {
private final BitSet bitSet;
public DocIdBitSet(BitSet bitSet) {
this.bitSet = bitSet;
@ -36,6 +36,11 @@ public class DocIdBitSet extends DocIdSet {
return new DocIdBitSetIterator(bitSet);
}
@Override
public Bits bits() {
return this;
}
/** This DocIdSet implementation is cacheable. */
@Override
public boolean isCacheable() {
@ -46,7 +51,18 @@ public class DocIdBitSet extends DocIdSet {
* Returns the underlying BitSet.
*/
public BitSet getBitSet() {
return this.bitSet;
return this.bitSet;
}
@Override
public boolean get(int index) {
return bitSet.get(index);
}
@Override
public int length() {
// the size may not be correct...
return bitSet.size();
}
private static class DocIdBitSetIterator extends DocIdSetIterator {

View File

@ -66,6 +66,11 @@ public final class FixedBitSet extends DocIdSet implements Bits {
return new OpenBitSetIterator(bits, bits.length);
}
@Override
public Bits bits() {
return this;
}
@Override
public int length() {
return numBits;

View File

@ -119,6 +119,11 @@ public class OpenBitSet extends DocIdSet implements Bits, Cloneable {
return new OpenBitSetIterator(bits, wlen);
}
@Override
public Bits bits() {
return this;
}
/** This DocIdSet implementation is cacheable. */
@Override
public boolean isCacheable() {

View File

@ -17,6 +17,7 @@ package org.apache.lucene.search;
* limitations under the License.
*/
import java.util.Random;
import java.util.concurrent.ExecutorService;
import java.io.IOException;
@ -31,20 +32,25 @@ import org.apache.lucene.util.Bits;
* TODO: Extend this by more checks, that's just a start.
*/
public class AssertingIndexSearcher extends IndexSearcher {
public AssertingIndexSearcher(IndexReader r) {
final Random random;
public AssertingIndexSearcher(Random random, IndexReader r) {
super(r);
this.random = new Random(random.nextLong());
}
public AssertingIndexSearcher(ReaderContext context) {
public AssertingIndexSearcher(Random random, ReaderContext context) {
super(context);
this.random = new Random(random.nextLong());
}
public AssertingIndexSearcher(IndexReader r, ExecutorService ex) {
public AssertingIndexSearcher(Random random, IndexReader r, ExecutorService ex) {
super(r, ex);
this.random = new Random(random.nextLong());
}
public AssertingIndexSearcher(ReaderContext context, ExecutorService ex) {
public AssertingIndexSearcher(Random random, ReaderContext context, ExecutorService ex) {
super(context, ex);
this.random = new Random(random.nextLong());
}
/** Ensures, that the returned {@code Weight} is not normalized again, which may produce wrong scores. */
@ -84,4 +90,16 @@ public class AssertingIndexSearcher extends IndexSearcher {
}
};
}
@Override
protected Query wrapFilter(Query query, Filter filter) {
if (random.nextBoolean())
return super.wrapFilter(query, filter);
return (filter == null) ? query : new FilteredQuery(query, filter) {
@Override
protected boolean useRandomAccess(Bits bits, int firstFilterDoc) {
return random.nextBoolean();
}
};
}
}

View File

@ -22,6 +22,7 @@ import java.io.IOException;
import junit.framework.Assert;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.util.Bits;
/**
* A unit test helper class to test when the filter is getting cached and when it is not.
@ -42,10 +43,10 @@ public class CachingWrapperFilterHelper extends CachingWrapperFilter {
}
@Override
public synchronized DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
public synchronized DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
final int saveMissCount = missCount;
DocIdSet docIdSet = super.getDocIdSet(context);
DocIdSet docIdSet = super.getDocIdSet(context, acceptDocs);
if (shouldHaveCache) {
Assert.assertEquals("Cache should have data ", saveMissCount, missCount);

View File

@ -1300,7 +1300,7 @@ public abstract class LuceneTestCase extends Assert {
if (maybeWrap && rarely()) {
r = new SlowMultiReaderWrapper(r);
}
IndexSearcher ret = random.nextBoolean() ? new AssertingIndexSearcher(r) : new AssertingIndexSearcher(r.getTopReaderContext());
IndexSearcher ret = random.nextBoolean() ? new AssertingIndexSearcher(random, r) : new AssertingIndexSearcher(random, r.getTopReaderContext());
ret.setSimilarityProvider(similarityProvider);
return ret;
} else {
@ -1312,13 +1312,13 @@ public abstract class LuceneTestCase extends Assert {
System.out.println("NOTE: newSearcher using ExecutorService with " + threads + " threads");
}
IndexSearcher ret = random.nextBoolean() ?
new AssertingIndexSearcher(r, ex) {
new AssertingIndexSearcher(random, r, ex) {
@Override
public void close() throws IOException {
super.close();
shutdownExecutorService(ex);
}
} : new AssertingIndexSearcher(r.getTopReaderContext(), ex) {
} : new AssertingIndexSearcher(random, r.getTopReaderContext(), ex) {
@Override
public void close() throws IOException {
super.close();

View File

@ -24,6 +24,7 @@ import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.OpenBitSet;
@ -109,7 +110,7 @@ public final class FieldCacheRewriteMethod extends MultiTermQuery.RewriteMethod
* results.
*/
@Override
public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
public DocIdSet getDocIdSet(AtomicReaderContext context, final Bits acceptDocs) throws IOException {
final FieldCache.DocTermsIndex fcsi = FieldCache.DEFAULT.getTermsIndex(context.reader, query.field);
// Cannot use FixedBitSet because we require long index (ord):
final OpenBitSet termSet = new OpenBitSet(fcsi.numOrd());
@ -163,7 +164,8 @@ public final class FieldCacheRewriteMethod extends MultiTermQuery.RewriteMethod
return DocIdSet.EMPTY_DOCIDSET;
}
return new FieldCacheRangeFilter.FieldCacheDocIdSet(context.reader, true) {
final int maxDoc = context.reader.maxDoc();
return new FieldCacheRangeFilter.FieldCacheDocIdSet(maxDoc, acceptDocs) {
@Override
boolean matchDoc(int doc) throws ArrayIndexOutOfBoundsException {
return termSet.get(fcsi.getOrd(doc));

View File

@ -160,7 +160,7 @@ final class JustCompileSearch {
// still added here in case someone will add abstract methods in the future.
@Override
public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
return null;
}
}
@ -288,12 +288,12 @@ final class JustCompileSearch {
static final class JustCompileSpanFilter extends SpanFilter {
@Override
public SpanFilterResult bitSpans(AtomicReaderContext context) throws IOException {
public SpanFilterResult bitSpans(AtomicReaderContext context, Bits acceptDocs) throws IOException {
throw new UnsupportedOperationException(UNSUPPORTED_MSG);
}
@Override
public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
return null;
}
}

View File

@ -19,15 +19,16 @@ package org.apache.lucene.search;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.util.DocIdBitSet;
import java.util.BitSet;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.Bits;
public class MockFilter extends Filter {
private boolean wasCalled;
@Override
public DocIdSet getDocIdSet(AtomicReaderContext context) {
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) {
wasCalled = true;
return new DocIdBitSet(new BitSet());
return new FixedBitSet(context.reader.maxDoc());
}
public void clear() {

View File

@ -18,9 +18,9 @@ package org.apache.lucene.search;
*/
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.util.DocIdBitSet;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.FixedBitSet;
import java.util.BitSet;
import java.io.IOException;
public class SingleDocTestFilter extends Filter {
@ -31,9 +31,10 @@ public class SingleDocTestFilter extends Filter {
}
@Override
public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
BitSet bits = new BitSet(context.reader.maxDoc());
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
FixedBitSet bits = new FixedBitSet(context.reader.maxDoc());
bits.set(doc);
return new DocIdBitSet(bits);
if (acceptDocs != null && !acceptDocs.get(doc)) bits.clear(doc);
return bits;
}
}

View File

@ -69,33 +69,21 @@ public class TestCachingSpanFilter extends LuceneTestCase {
final SpanFilter startFilter = new SpanQueryFilter(new SpanTermQuery(new Term("id", "1")));
// ignore deletions
CachingSpanFilter filter = new CachingSpanFilter(startFilter, CachingWrapperFilter.DeletesMode.IGNORE);
CachingSpanFilter filter = new CachingSpanFilter(startFilter);
docs = searcher.search(new MatchAllDocsQuery(), filter, 1);
assertEquals("[query + filter] Should find a hit...", 1, docs.totalHits);
ConstantScoreQuery constantScore = new ConstantScoreQuery(filter);
int missCount = filter.missCount;
assertTrue(missCount > 0);
Query constantScore = new ConstantScoreQuery(filter);
docs = searcher.search(constantScore, 1);
assertEquals("[just filter] Should find a hit...", 1, docs.totalHits);
assertEquals(missCount, filter.missCount);
// now delete the doc, refresh the reader, and see that
// it's not there
_TestUtil.keepFullyDeletedSegments(writer.w);
writer.deleteDocuments(new Term("id", "1"));
reader = refreshReader(reader);
searcher.close();
searcher = newSearcher(reader, false);
docs = searcher.search(new MatchAllDocsQuery(), filter, 1);
assertEquals("[query + filter] Should *not* find a hit...", 0, docs.totalHits);
docs = searcher.search(constantScore, 1);
assertEquals("[just filter] Should find a hit...", 1, docs.totalHits);
// force cache to regenerate:
filter = new CachingSpanFilter(startFilter, CachingWrapperFilter.DeletesMode.RECACHE);
// NOTE: important to hold ref here so GC doesn't clear
// the cache entry! Else the assert below may sometimes
// fail:
IndexReader oldReader = reader;
writer.addDocument(doc);
reader = refreshReader(reader);
@ -103,27 +91,19 @@ public class TestCachingSpanFilter extends LuceneTestCase {
searcher = newSearcher(reader, false);
docs = searcher.search(new MatchAllDocsQuery(), filter, 1);
assertEquals("[query + filter] Should find a hit...", 1, docs.totalHits);
assertEquals("[query + filter] Should find 2 hits...", 2, docs.totalHits);
assertTrue(filter.missCount > missCount);
missCount = filter.missCount;
constantScore = new ConstantScoreQuery(filter);
docs = searcher.search(constantScore, 1);
assertEquals("[just filter] Should find a hit...", 1, docs.totalHits);
assertEquals("[just filter] Should find a hit...", 2, docs.totalHits);
assertEquals(missCount, filter.missCount);
// NOTE: important to hold ref here so GC doesn't clear
// the cache entry! Else the assert below may sometimes
// fail:
IndexReader oldReader = reader;
// make sure we get a cache hit when we reopen readers
// that had no new deletions
// Deletes nothing:
writer.deleteDocuments(new Term("foo", "bar"));
reader = refreshReader(reader);
assertTrue(reader == oldReader);
int missCount = filter.missCount;
docs = searcher.search(constantScore, 1);
assertEquals("[just filter] Should find a hit...", 1, docs.totalHits);
assertEquals(missCount, filter.missCount);
IndexReader oldReader2 = reader;
// now delete the doc, refresh the reader, and see that it's not there
writer.deleteDocuments(new Term("id", "1"));
@ -134,15 +114,18 @@ public class TestCachingSpanFilter extends LuceneTestCase {
docs = searcher.search(new MatchAllDocsQuery(), filter, 1);
assertEquals("[query + filter] Should *not* find a hit...", 0, docs.totalHits);
assertEquals(missCount, filter.missCount);
docs = searcher.search(constantScore, 1);
assertEquals("[just filter] Should *not* find a hit...", 0, docs.totalHits);
assertEquals(missCount, filter.missCount);
// NOTE: silliness to make sure JRE does not optimize
// away our holding onto oldReader to prevent
// CachingWrapperFilter's WeakHashMap from dropping the
// entry:
assertTrue(oldReader != null);
assertTrue(oldReader2 != null);
searcher.close();
writer.close();
@ -160,4 +143,5 @@ public class TestCachingSpanFilter extends LuceneTestCase {
return oldReader;
}
}
}

View File

@ -29,12 +29,13 @@ import org.apache.lucene.index.SerialMergeScheduler;
import org.apache.lucene.index.SlowMultiReaderWrapper;
import org.apache.lucene.index.Term;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util._TestUtil;
public class TestCachingWrapperFilter extends LuceneTestCase {
public void testCachingWorks() throws Exception {
Directory dir = newDirectory();
RandomIndexWriter writer = new RandomIndexWriter(random, dir);
@ -46,15 +47,15 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
CachingWrapperFilter cacher = new CachingWrapperFilter(filter);
// first time, nested filter is called
cacher.getDocIdSet(context);
cacher.getDocIdSet(context, context.reader.getLiveDocs());
assertTrue("first time", filter.wasCalled());
// make sure no exception if cache is holding the wrong docIdSet
cacher.getDocIdSet(context);
cacher.getDocIdSet(context, context.reader.getLiveDocs());
// second time, nested filter should not be called
filter.clear();
cacher.getDocIdSet(context);
cacher.getDocIdSet(context, context.reader.getLiveDocs());
assertFalse("second time", filter.wasCalled());
reader.close();
@ -71,14 +72,14 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
final Filter filter = new Filter() {
@Override
public DocIdSet getDocIdSet(AtomicReaderContext context) {
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) {
return null;
}
};
CachingWrapperFilter cacher = new CachingWrapperFilter(filter);
// the caching filter should return the empty set constant
assertSame(DocIdSet.EMPTY_DOCIDSET, cacher.getDocIdSet(context));
assertSame(DocIdSet.EMPTY_DOCIDSET, cacher.getDocIdSet(context, context.reader.getLiveDocs()));
reader.close();
dir.close();
@ -94,7 +95,7 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
final Filter filter = new Filter() {
@Override
public DocIdSet getDocIdSet(AtomicReaderContext context) {
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) {
return new DocIdSet() {
@Override
public DocIdSetIterator iterator() {
@ -106,7 +107,7 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
CachingWrapperFilter cacher = new CachingWrapperFilter(filter);
// the caching filter should return the empty set constant
assertSame(DocIdSet.EMPTY_DOCIDSET, cacher.getDocIdSet(context));
assertSame(DocIdSet.EMPTY_DOCIDSET, cacher.getDocIdSet(context, context.reader.getLiveDocs()));
reader.close();
dir.close();
@ -116,8 +117,8 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
assertTrue(reader.getTopReaderContext().isAtomic);
AtomicReaderContext context = (AtomicReaderContext) reader.getTopReaderContext();
final CachingWrapperFilter cacher = new CachingWrapperFilter(filter);
final DocIdSet originalSet = filter.getDocIdSet(context);
final DocIdSet cachedSet = cacher.getDocIdSet(context);
final DocIdSet originalSet = filter.getDocIdSet(context, context.reader.getLiveDocs());
final DocIdSet cachedSet = cacher.getDocIdSet(context, context.reader.getLiveDocs());
assertTrue(cachedSet.isCacheable());
assertEquals(shouldCacheable, originalSet.isCacheable());
//System.out.println("Original: "+originalSet.getClass().getName()+" -- cached: "+cachedSet.getClass().getName());
@ -145,7 +146,7 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
// a fixedbitset filter is always cacheable
assertDocIdSetCacheable(reader, new Filter() {
@Override
public DocIdSet getDocIdSet(AtomicReaderContext context) {
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) {
return new FixedBitSet(context.reader.maxDoc());
}
}, true);
@ -187,90 +188,41 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
final Filter startFilter = new QueryWrapperFilter(new TermQuery(new Term("id", "1")));
// ignore deletions
CachingWrapperFilter filter = new CachingWrapperFilter(startFilter, CachingWrapperFilter.DeletesMode.IGNORE);
CachingWrapperFilter filter = new CachingWrapperFilter(startFilter);
docs = searcher.search(new MatchAllDocsQuery(), filter, 1);
assertEquals("[query + filter] Should find a hit...", 1, docs.totalHits);
ConstantScoreQuery constantScore = new ConstantScoreQuery(filter);
docs = searcher.search(constantScore, 1);
assertEquals("[just filter] Should find a hit...", 1, docs.totalHits);
// now delete the doc, refresh the reader, and see that it's not there
_TestUtil.keepFullyDeletedSegments(writer.w);
writer.deleteDocuments(new Term("id", "1"));
reader = refreshReader(reader);
searcher.close();
searcher = newSearcher(reader, false);
docs = searcher.search(new MatchAllDocsQuery(), filter, 1);
assertEquals("[query + filter] Should *not* find a hit...", 0, docs.totalHits);
docs = searcher.search(constantScore, 1);
assertEquals("[just filter] Should find a hit...", 1, docs.totalHits);
// force cache to regenerate:
filter = new CachingWrapperFilter(startFilter, CachingWrapperFilter.DeletesMode.RECACHE);
writer.addDocument(doc);
reader = refreshReader(reader);
searcher.close();
searcher = newSearcher(reader, false);
docs = searcher.search(new MatchAllDocsQuery(), filter, 1);
assertEquals("[query + filter] Should find a hit...", 1, docs.totalHits);
constantScore = new ConstantScoreQuery(filter);
int missCount = filter.missCount;
assertTrue(missCount > 0);
Query constantScore = new ConstantScoreQuery(filter);
docs = searcher.search(constantScore, 1);
assertEquals("[just filter] Should find a hit...", 1, docs.totalHits);
assertEquals(missCount, filter.missCount);
// NOTE: important to hold ref here so GC doesn't clear
// the cache entry! Else the assert below may sometimes
// fail:
IndexReader oldReader = reader;
// make sure we get a cache hit when we reopen reader
// that had no change to deletions
writer.deleteDocuments(new Term("foo", "bar"));
reader = refreshReader(reader);
assertTrue(reader == oldReader);
int missCount = filter.missCount;
docs = searcher.search(constantScore, 1);
assertEquals("[just filter] Should find a hit...", 1, docs.totalHits);
assertEquals(missCount, filter.missCount);
// now delete the doc, refresh the reader, and see that it's not there
writer.deleteDocuments(new Term("id", "1"));
reader = refreshReader(reader);
searcher.close();
searcher = newSearcher(reader, false);
missCount = filter.missCount;
docs = searcher.search(new MatchAllDocsQuery(), filter, 1);
assertEquals(missCount+1, filter.missCount);
assertEquals("[query + filter] Should *not* find a hit...", 0, docs.totalHits);
docs = searcher.search(constantScore, 1);
assertEquals("[just filter] Should *not* find a hit...", 0, docs.totalHits);
// apply deletions dynamically
filter = new CachingWrapperFilter(startFilter, CachingWrapperFilter.DeletesMode.DYNAMIC);
writer.addDocument(doc);
reader = refreshReader(reader);
searcher.close();
searcher = newSearcher(reader, false);
docs = searcher.search(new MatchAllDocsQuery(), filter, 1);
assertEquals("[query + filter] Should find a hit...", 1, docs.totalHits);
assertEquals("[query + filter] Should find 2 hits...", 2, docs.totalHits);
assertTrue(filter.missCount > missCount);
missCount = filter.missCount;
constantScore = new ConstantScoreQuery(filter);
docs = searcher.search(constantScore, 1);
assertEquals("[just filter] Should find a hit...", 1, docs.totalHits);
assertEquals("[just filter] Should find a hit...", 2, docs.totalHits);
assertEquals(missCount, filter.missCount);
// NOTE: important to hold ref here so GC doesn't clear
// the cache entry! Else the assert below may sometimes
// fail:
IndexReader oldReader2 = reader;
// now delete the doc, refresh the reader, and see that it's not there
writer.deleteDocuments(new Term("id", "1"));
@ -281,12 +233,10 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
docs = searcher.search(new MatchAllDocsQuery(), filter, 1);
assertEquals("[query + filter] Should *not* find a hit...", 0, docs.totalHits);
assertEquals(missCount, filter.missCount);
missCount = filter.missCount;
docs = searcher.search(constantScore, 1);
assertEquals("[just filter] Should *not* find a hit...", 0, docs.totalHits);
// doesn't count as a miss
assertEquals(missCount, filter.missCount);
// NOTE: silliness to make sure JRE does not optimize
@ -294,6 +244,7 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
// CachingWrapperFilter's WeakHashMap from dropping the
// entry:
assertTrue(oldReader != null);
assertTrue(oldReader2 != null);
searcher.close();
reader.close();
@ -311,4 +262,5 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
return oldReader;
}
}
}

View File

@ -130,5 +130,32 @@ public class TestConstantScoreQuery extends LuceneTestCase {
if (directory != null) directory.close();
}
}
public void testConstantScoreQueryAndFilter() throws Exception {
Directory d = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random, d);
Document doc = new Document();
doc.add(newField("field", "a", StringField.TYPE_UNSTORED));
w.addDocument(doc);
doc = new Document();
doc.add(newField("field", "b", StringField.TYPE_UNSTORED));
w.addDocument(doc);
IndexReader r = w.getReader();
w.close();
Filter filterB = new CachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("field", "b"))));
Query query = new ConstantScoreQuery(filterB);
IndexSearcher s = new IndexSearcher(r);
assertEquals(1, s.search(query, filterB, 1).totalHits); // Query for field:b, Filter field:b
Filter filterA = new CachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("field", "a"))));
query = new ConstantScoreQuery(filterA);
assertEquals(0, s.search(query, filterB, 1).totalHits); // Query field:b, Filter field:a
r.close();
d.close();
}
}

View File

@ -30,6 +30,7 @@ import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.LuceneTestCase;
public class TestDocIdSet extends LuceneTestCase {
@ -114,7 +115,7 @@ public class TestDocIdSet extends LuceneTestCase {
// Now search w/ a Filter which returns a null DocIdSet
Filter f = new Filter() {
@Override
public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
return null;
}
};

View File

@ -28,6 +28,7 @@ import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.DocIdBitSet;
import org.apache.lucene.util.LuceneTestCase;
@ -82,6 +83,7 @@ public class TestFilteredQuery extends LuceneTestCase {
writer.close ();
searcher = newSearcher(reader);
query = new TermQuery (new Term ("field", "three"));
filter = newStaticFilterB();
}
@ -90,10 +92,11 @@ public class TestFilteredQuery extends LuceneTestCase {
private static Filter newStaticFilterB() {
return new Filter() {
@Override
public DocIdSet getDocIdSet (AtomicReaderContext context) {
public DocIdSet getDocIdSet (AtomicReaderContext context, Bits acceptDocs) {
if (acceptDocs == null) acceptDocs = new Bits.MatchAllBits(5);
BitSet bitset = new BitSet(5);
bitset.set (1);
bitset.set (3);
if (acceptDocs.get(1)) bitset.set(1);
if (acceptDocs.get(3)) bitset.set(3);
return new DocIdBitSet(bitset);
}
};
@ -106,10 +109,16 @@ public class TestFilteredQuery extends LuceneTestCase {
directory.close();
super.tearDown();
}
public void testFilteredQuery() throws Exception {
// force the filter to be executed as bits
tFilteredQuery(true);
// force the filter to be executed as iterator
tFilteredQuery(false);
}
public void testFilteredQuery()
throws Exception {
Query filteredquery = new FilteredQuery (query, filter);
private void tFilteredQuery(final boolean useRandomAccess) throws Exception {
Query filteredquery = new FilteredQueryRA(query, filter, useRandomAccess);
ScoreDoc[] hits = searcher.search (filteredquery, null, 1000).scoreDocs;
assertEquals (1, hits.length);
assertEquals (1, hits[0].doc);
@ -119,18 +128,18 @@ public class TestFilteredQuery extends LuceneTestCase {
assertEquals (1, hits.length);
assertEquals (1, hits[0].doc);
filteredquery = new FilteredQuery (new TermQuery (new Term ("field", "one")), filter);
filteredquery = new FilteredQueryRA(new TermQuery (new Term ("field", "one")), filter, useRandomAccess);
hits = searcher.search (filteredquery, null, 1000).scoreDocs;
assertEquals (2, hits.length);
QueryUtils.check(random, filteredquery,searcher);
filteredquery = new FilteredQuery (new TermQuery (new Term ("field", "x")), filter);
filteredquery = new FilteredQueryRA(new TermQuery (new Term ("field", "x")), filter, useRandomAccess);
hits = searcher.search (filteredquery, null, 1000).scoreDocs;
assertEquals (1, hits.length);
assertEquals (3, hits[0].doc);
QueryUtils.check(random, filteredquery,searcher);
filteredquery = new FilteredQuery (new TermQuery (new Term ("field", "y")), filter);
filteredquery = new FilteredQueryRA(new TermQuery (new Term ("field", "y")), filter, useRandomAccess);
hits = searcher.search (filteredquery, null, 1000).scoreDocs;
assertEquals (0, hits.length);
QueryUtils.check(random, filteredquery,searcher);
@ -147,7 +156,7 @@ public class TestFilteredQuery extends LuceneTestCase {
BooleanQuery bq2 = new BooleanQuery();
tq = new TermQuery (new Term ("field", "one"));
filteredquery = new FilteredQuery(tq, f);
filteredquery = new FilteredQueryRA(tq, f, useRandomAccess);
filteredquery.setBoost(boost);
bq2.add(filteredquery, Occur.MUST);
bq2.add(new TermQuery (new Term ("field", "five")), Occur.MUST);
@ -161,7 +170,8 @@ public class TestFilteredQuery extends LuceneTestCase {
private static Filter newStaticFilterA() {
return new Filter() {
@Override
public DocIdSet getDocIdSet (AtomicReaderContext context) {
public DocIdSet getDocIdSet (AtomicReaderContext context, Bits acceptDocs) {
assertNull("acceptDocs should be null, as we have an index without deletions", acceptDocs);
BitSet bitset = new BitSet(5);
bitset.set(0, 5);
return new DocIdBitSet(bitset);
@ -187,40 +197,112 @@ public class TestFilteredQuery extends LuceneTestCase {
* This tests FilteredQuery's rewrite correctness
*/
public void testRangeQuery() throws Exception {
// force the filter to be executed as bits
tRangeQuery(true);
tRangeQuery(false);
}
private void tRangeQuery(final boolean useRandomAccess) throws Exception {
TermRangeQuery rq = TermRangeQuery.newStringRange(
"sorter", "b", "d", true, true);
Query filteredquery = new FilteredQuery(rq, filter);
Query filteredquery = new FilteredQueryRA(rq, filter, useRandomAccess);
ScoreDoc[] hits = searcher.search(filteredquery, null, 1000).scoreDocs;
assertEquals(2, hits.length);
QueryUtils.check(random, filteredquery,searcher);
}
public void testBoolean() throws Exception {
public void testBooleanMUST() throws Exception {
// force the filter to be executed as bits
tBooleanMUST(true);
// force the filter to be executed as iterator
tBooleanMUST(false);
}
private void tBooleanMUST(final boolean useRandomAccess) throws Exception {
BooleanQuery bq = new BooleanQuery();
Query query = new FilteredQuery(new MatchAllDocsQuery(),
new SingleDocTestFilter(0));
Query query = new FilteredQueryRA(new MatchAllDocsQuery(), new SingleDocTestFilter(0), useRandomAccess);
bq.add(query, BooleanClause.Occur.MUST);
query = new FilteredQuery(new MatchAllDocsQuery(),
new SingleDocTestFilter(1));
query = new FilteredQueryRA(new MatchAllDocsQuery(), new SingleDocTestFilter(1), useRandomAccess);
bq.add(query, BooleanClause.Occur.MUST);
ScoreDoc[] hits = searcher.search(bq, null, 1000).scoreDocs;
assertEquals(0, hits.length);
QueryUtils.check(random, query,searcher);
}
public void testBooleanSHOULD() throws Exception {
// force the filter to be executed as bits
tBooleanSHOULD(true);
// force the filter to be executed as iterator
tBooleanSHOULD(false);
}
private void tBooleanSHOULD(final boolean useRandomAccess) throws Exception {
BooleanQuery bq = new BooleanQuery();
Query query = new FilteredQueryRA(new MatchAllDocsQuery(), new SingleDocTestFilter(0), useRandomAccess);
bq.add(query, BooleanClause.Occur.SHOULD);
query = new FilteredQueryRA(new MatchAllDocsQuery(), new SingleDocTestFilter(1), useRandomAccess);
bq.add(query, BooleanClause.Occur.SHOULD);
ScoreDoc[] hits = searcher.search(bq, null, 1000).scoreDocs;
assertEquals(2, hits.length);
QueryUtils.check(random, query,searcher);
}
// Make sure BooleanQuery, which does out-of-order
// scoring, inside FilteredQuery, works
public void testBoolean2() throws Exception {
// force the filter to be executed as bits
tBoolean2(true);
// force the filter to be executed as iterator
tBoolean2(false);
}
private void tBoolean2(final boolean useRandomAccess) throws Exception {
BooleanQuery bq = new BooleanQuery();
Query query = new FilteredQuery(bq,
new SingleDocTestFilter(0));
Query query = new FilteredQueryRA(bq, new SingleDocTestFilter(0), useRandomAccess);
bq.add(new TermQuery(new Term("field", "one")), BooleanClause.Occur.SHOULD);
bq.add(new TermQuery(new Term("field", "two")), BooleanClause.Occur.SHOULD);
ScoreDoc[] hits = searcher.search(query, 1000).scoreDocs;
assertEquals(1, hits.length);
QueryUtils.check(random, query, searcher);
}
public void testChainedFilters() throws Exception {
// force the filter to be executed as bits
tChainedFilters(true);
// force the filter to be executed as iterator
tChainedFilters(false);
}
private void tChainedFilters(final boolean useRandomAccess) throws Exception {
Query query = new TestFilteredQuery.FilteredQueryRA(new TestFilteredQuery.FilteredQueryRA(
new MatchAllDocsQuery(), new CachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("field", "three")))), useRandomAccess),
new CachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("field", "four")))), useRandomAccess);
ScoreDoc[] hits = searcher.search(query, 10).scoreDocs;
assertEquals(2, hits.length);
QueryUtils.check(random, query, searcher);
// one more:
query = new TestFilteredQuery.FilteredQueryRA(query,
new CachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("field", "five")))), useRandomAccess);
hits = searcher.search(query, 10).scoreDocs;
assertEquals(1, hits.length);
QueryUtils.check(random, query, searcher);
}
public static final class FilteredQueryRA extends FilteredQuery {
private final boolean useRandomAccess;
public FilteredQueryRA(Query q, Filter f, boolean useRandomAccess) {
super(q,f);
this.useRandomAccess = useRandomAccess;
}
@Override
protected boolean useRandomAccess(Bits bits, int firstFilterDoc) {
return useRandomAccess;
}
}
}

View File

@ -30,6 +30,7 @@ import org.apache.lucene.index.Term;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.LockObtainFailedException;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.FixedBitSet;
@ -95,7 +96,8 @@ public class TestFilteredSearch extends LuceneTestCase {
}
@Override
public DocIdSet getDocIdSet(AtomicReaderContext context) {
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) {
assertNull("acceptDocs should be null, as we have an index without deletions", acceptDocs);
assert context.isAtomic;
final FixedBitSet set = new FixedBitSet(context.reader.maxDoc());
int docBase = context.docBase;

View File

@ -182,13 +182,13 @@ public class TestNumericRangeQuery32 extends LuceneTestCase {
public void testInverseRange() throws Exception {
AtomicReaderContext context = (AtomicReaderContext) new SlowMultiReaderWrapper(searcher.getIndexReader()).getTopReaderContext();
NumericRangeFilter<Integer> f = NumericRangeFilter.newIntRange("field8", 8, 1000, -1000, true, true);
assertSame("A inverse range should return the EMPTY_DOCIDSET instance", DocIdSet.EMPTY_DOCIDSET, f.getDocIdSet(context));
assertSame("A inverse range should return the EMPTY_DOCIDSET instance", DocIdSet.EMPTY_DOCIDSET, f.getDocIdSet(context, context.reader.getLiveDocs()));
f = NumericRangeFilter.newIntRange("field8", 8, Integer.MAX_VALUE, null, false, false);
assertSame("A exclusive range starting with Integer.MAX_VALUE should return the EMPTY_DOCIDSET instance",
DocIdSet.EMPTY_DOCIDSET, f.getDocIdSet(context));
DocIdSet.EMPTY_DOCIDSET, f.getDocIdSet(context, context.reader.getLiveDocs()));
f = NumericRangeFilter.newIntRange("field8", 8, null, Integer.MIN_VALUE, false, false);
assertSame("A exclusive range ending with Integer.MIN_VALUE should return the EMPTY_DOCIDSET instance",
DocIdSet.EMPTY_DOCIDSET, f.getDocIdSet(context));
DocIdSet.EMPTY_DOCIDSET, f.getDocIdSet(context, context.reader.getLiveDocs()));
}
@Test

View File

@ -188,13 +188,13 @@ public class TestNumericRangeQuery64 extends LuceneTestCase {
AtomicReaderContext context = (AtomicReaderContext) new SlowMultiReaderWrapper(searcher.getIndexReader()).getTopReaderContext();
NumericRangeFilter<Long> f = NumericRangeFilter.newLongRange("field8", 8, 1000L, -1000L, true, true);
assertSame("A inverse range should return the EMPTY_DOCIDSET instance", DocIdSet.EMPTY_DOCIDSET,
f.getDocIdSet(context));
f.getDocIdSet(context, context.reader.getLiveDocs()));
f = NumericRangeFilter.newLongRange("field8", 8, Long.MAX_VALUE, null, false, false);
assertSame("A exclusive range starting with Long.MAX_VALUE should return the EMPTY_DOCIDSET instance",
DocIdSet.EMPTY_DOCIDSET, f.getDocIdSet(context));
DocIdSet.EMPTY_DOCIDSET, f.getDocIdSet(context, context.reader.getLiveDocs()));
f = NumericRangeFilter.newLongRange("field8", 8, null, Long.MIN_VALUE, false, false);
assertSame("A exclusive range ending with Long.MIN_VALUE should return the EMPTY_DOCIDSET instance",
DocIdSet.EMPTY_DOCIDSET, f.getDocIdSet(context));
DocIdSet.EMPTY_DOCIDSET, f.getDocIdSet(context, context.reader.getLiveDocs()));
}
@Test

View File

@ -1,5 +1,6 @@
package org.apache.lucene.search;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.DocIdBitSet;
import org.apache.lucene.util.LuceneTestCase;
@ -141,7 +142,8 @@ public class TestScorerPerf extends LuceneTestCase {
final BitSet rnd = sets[random.nextInt(sets.length)];
Query q = new ConstantScoreQuery(new Filter() {
@Override
public DocIdSet getDocIdSet(AtomicReaderContext context) {
public DocIdSet getDocIdSet (AtomicReaderContext context, Bits acceptDocs) {
assertNull("acceptDocs should be null, as we have an index without deletions", acceptDocs);
return new DocIdBitSet(rnd);
}
});

View File

@ -53,6 +53,7 @@ import org.apache.lucene.search.cache.LongValuesCreator;
import org.apache.lucene.search.cache.ShortValuesCreator;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.LockObtainFailedException;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.DocIdBitSet;
import org.apache.lucene.util.LuceneTestCase;
@ -730,7 +731,8 @@ public class TestSort extends LuceneTestCase {
// a filter that only allows through the first hit
Filter filt = new Filter() {
@Override
public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
public DocIdSet getDocIdSet (AtomicReaderContext context, Bits acceptDocs) {
assertNull("acceptDocs should be null, as we have no deletions", acceptDocs);
BitSet bs = new BitSet(context.reader.maxDoc());
bs.set(0, context.reader.maxDoc());
bs.set(docs1.scoreDocs[0].doc);

View File

@ -49,7 +49,7 @@ public class TestSpanQueryFilter extends LuceneTestCase {
int subIndex = ReaderUtil.subIndex(number, leaves); // find the reader with this document in it
SpanTermQuery query = new SpanTermQuery(new Term("field", English.intToEnglish(number).trim()));
SpanQueryFilter filter = new SpanQueryFilter(query);
SpanFilterResult result = filter.bitSpans(leaves[subIndex]);
SpanFilterResult result = filter.bitSpans(leaves[subIndex], leaves[subIndex].reader.getLiveDocs());
DocIdSet docIdSet = result.getDocIdSet();
assertTrue("docIdSet is null and it shouldn't be", docIdSet != null);
assertContainsDocId("docIdSet doesn't contain docId 10", docIdSet, number - leaves[subIndex].docBase);

View File

@ -44,11 +44,11 @@ public class TestTopDocsMerge extends LuceneTestCase {
}
public void search(Weight weight, Collector collector) throws IOException {
search(ctx, weight, null, collector);
search(ctx, weight, collector);
}
public TopDocs search(Weight weight, int topN) throws IOException {
return search(ctx, weight, null, null, topN);
return search(ctx, weight, null, topN);
}
@Override

View File

@ -505,7 +505,7 @@ public class BlockGroupingCollector extends Collector {
subDocUpto = 0;
docBase = readerContext.docBase;
//System.out.println("setNextReader base=" + docBase + " r=" + readerContext.reader);
lastDocPerGroupBits = lastDocPerGroup.getDocIdSet(readerContext).iterator();
lastDocPerGroupBits = lastDocPerGroup.getDocIdSet(readerContext, readerContext.reader.getLiveDocs()).iterator();
groupEndDocID = -1;
currentReaderContext = readerContext;

View File

@ -1221,11 +1221,11 @@ public class TestGrouping extends LuceneTestCase {
}
public void search(Weight weight, Collector collector) throws IOException {
search(ctx, weight, null, collector);
search(ctx, weight, collector);
}
public TopDocs search(Weight weight, int topN) throws IOException {
return search(ctx, weight, null, null, topN);
return search(ctx, weight, null, topN);
}
@Override

View File

@ -163,7 +163,7 @@ public class BlockJoinQuery extends Query {
return null;
}
final DocIdSet parents = parentsFilter.getDocIdSet(readerContext);
final DocIdSet parents = parentsFilter.getDocIdSet(readerContext, readerContext.reader.getLiveDocs());
// TODO: once we do random-access filters we can
// generalize this:
if (parents == null) {

View File

@ -24,10 +24,12 @@ import java.util.Iterator;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.search.BitsFilteredDocIdSet;
import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.Filter;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.FixedBitSet;
/**
@ -48,7 +50,7 @@ public class BooleanFilter extends Filter implements Iterable<FilterClause> {
* of the filters that have been added.
*/
@Override
public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
FixedBitSet res = null;
final IndexReader reader = context.reader;
@ -91,12 +93,13 @@ public class BooleanFilter extends Filter implements Iterable<FilterClause> {
}
}
return res != null ? res : DocIdSet.EMPTY_DOCIDSET;
return res != null ? BitsFilteredDocIdSet.wrap(res, acceptDocs) : DocIdSet.EMPTY_DOCIDSET;
}
private static DocIdSetIterator getDISI(Filter filter, AtomicReaderContext context)
throws IOException {
final DocIdSet set = filter.getDocIdSet(context);
// we dont pass acceptDocs, we will filter at the end using an additional filter
final DocIdSet set = filter.getDocIdSet(context, null);
return (set == null || set == DocIdSet.EMPTY_DOCIDSET) ? null : set.iterator();
}

View File

@ -19,9 +19,11 @@ package org.apache.lucene.queries;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.search.BitsFilteredDocIdSet;
import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.Filter;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.OpenBitSet;
import org.apache.lucene.util.OpenBitSetDISI;
@ -97,21 +99,22 @@ public class ChainedFilter extends Filter {
* {@link Filter#getDocIdSet}.
*/
@Override
public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
int[] index = new int[1]; // use array as reference to modifiable int;
index[0] = 0; // an object attribute would not be thread safe.
if (logic != -1) {
return getDocIdSet(context, logic, index);
return BitsFilteredDocIdSet.wrap(getDocIdSet(context, logic, index), acceptDocs);
} else if (logicArray != null) {
return getDocIdSet(context, logicArray, index);
return BitsFilteredDocIdSet.wrap(getDocIdSet(context, logicArray, index), acceptDocs);
}
return getDocIdSet(context, DEFAULT, index);
return BitsFilteredDocIdSet.wrap(getDocIdSet(context, DEFAULT, index), acceptDocs);
}
private DocIdSetIterator getDISI(Filter filter, AtomicReaderContext context)
throws IOException {
DocIdSet docIdSet = filter.getDocIdSet(context);
// we dont pass acceptDocs, we will filter at the end using an additional filter
DocIdSet docIdSet = filter.getDocIdSet(context, null);
if (docIdSet == null) {
return DocIdSet.EMPTY_DOCIDSET.iterator();
} else {
@ -156,7 +159,8 @@ public class ChainedFilter extends Filter {
throws IOException {
OpenBitSetDISI result = initialResult(context, logic, index);
for (; index[0] < chain.length; index[0]++) {
doChain(result, logic, chain[index[0]].getDocIdSet(context));
// we dont pass acceptDocs, we will filter at the end using an additional filter
doChain(result, logic, chain[index[0]].getDocIdSet(context, null));
}
return result;
}
@ -176,7 +180,8 @@ public class ChainedFilter extends Filter {
OpenBitSetDISI result = initialResult(context, logic[0], index);
for (; index[0] < chain.length; index[0]++) {
doChain(result, logic[index[0]], chain[index[0]].getDocIdSet(context));
// we dont pass acceptDocs, we will filter at the end using an additional filter
doChain(result, logic[index[0]], chain[index[0]].getDocIdSet(context, null));
}
return result;
}

View File

@ -54,7 +54,7 @@ public class TermsFilter extends Filter {
*/
@Override
public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
IndexReader reader = context.reader;
FixedBitSet result = new FixedBitSet(reader.maxDoc());
Fields fields = reader.fields();
@ -64,7 +64,6 @@ public class TermsFilter extends Filter {
}
BytesRef br = new BytesRef();
Bits liveDocs = reader.getLiveDocs();
String lastField = null;
Terms termsC = null;
TermsEnum termsEnum = null;
@ -82,7 +81,7 @@ public class TermsFilter extends Filter {
if (terms != null) { // TODO this check doesn't make sense, decide which variable its supposed to be for
br.copy(term.bytes());
if (termsEnum.seekCeil(br) == TermsEnum.SeekStatus.FOUND) {
docs = termsEnum.docs(liveDocs, docs);
docs = termsEnum.docs(acceptDocs, docs);
while (docs.nextDoc() != DocsEnum.NO_MORE_DOCS) {
result.set(docs.docID());
}

View File

@ -35,6 +35,7 @@ import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.LuceneTestCase;
import java.io.IOException;
@ -94,7 +95,7 @@ public class BooleanFilterTest extends LuceneTestCase {
private Filter getNullDISFilter() {
return new Filter() {
@Override
public DocIdSet getDocIdSet(AtomicReaderContext context) {
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) {
return null;
}
};
@ -103,7 +104,7 @@ public class BooleanFilterTest extends LuceneTestCase {
private Filter getNullDISIFilter() {
return new Filter() {
@Override
public DocIdSet getDocIdSet(AtomicReaderContext context) {
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) {
return new DocIdSet() {
@Override
public DocIdSetIterator iterator() {
@ -122,7 +123,7 @@ public class BooleanFilterTest extends LuceneTestCase {
private void tstFilterCard(String mes, int expected, Filter filt)
throws Exception {
// BooleanFilter never returns null DIS or null DISI!
DocIdSetIterator disi = filt.getDocIdSet(new AtomicReaderContext(reader)).iterator();
DocIdSetIterator disi = filt.getDocIdSet(new AtomicReaderContext(reader), reader.getLiveDocs()).iterator();
int actual = 0;
while (disi.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
actual++;

View File

@ -70,19 +70,19 @@ public class TermsFilterTest extends LuceneTestCase {
TermsFilter tf = new TermsFilter();
tf.addTerm(new Term(fieldName, "19"));
FixedBitSet bits = (FixedBitSet) tf.getDocIdSet(context);
FixedBitSet bits = (FixedBitSet) tf.getDocIdSet(context, context.reader.getLiveDocs());
assertEquals("Must match nothing", 0, bits.cardinality());
tf.addTerm(new Term(fieldName, "20"));
bits = (FixedBitSet) tf.getDocIdSet(context);
bits = (FixedBitSet) tf.getDocIdSet(context, context.reader.getLiveDocs());
assertEquals("Must match 1", 1, bits.cardinality());
tf.addTerm(new Term(fieldName, "10"));
bits = (FixedBitSet) tf.getDocIdSet(context);
bits = (FixedBitSet) tf.getDocIdSet(context, context.reader.getLiveDocs());
assertEquals("Must match 2", 2, bits.cardinality());
tf.addTerm(new Term(fieldName, "00"));
bits = (FixedBitSet) tf.getDocIdSet(context);
bits = (FixedBitSet) tf.getDocIdSet(context, context.reader.getLiveDocs());
assertEquals("Must match 2", 2, bits.cardinality());
reader.close();
@ -113,7 +113,7 @@ public class TermsFilterTest extends LuceneTestCase {
MultiReader multi = new MultiReader(reader1, reader2);
for (IndexReader.AtomicReaderContext context : ReaderUtil.leaves(multi.getTopReaderContext())) {
FixedBitSet bits = (FixedBitSet) tf.getDocIdSet(context);
FixedBitSet bits = (FixedBitSet) tf.getDocIdSet(context, context.reader.getLiveDocs());
assertTrue("Must be >= 0", bits.cardinality() >= 0);
}
multi.close();

View File

@ -21,6 +21,7 @@ import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.NumericRangeFilter;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.NumericUtils;
import org.apache.lucene.queryparser.xml.DOMUtils;
import org.apache.lucene.queryparser.xml.FilterBuilder;
@ -155,7 +156,7 @@ public class NumericRangeFilterBuilder implements FilterBuilder {
static class NoMatchFilter extends Filter {
@Override
public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
return null;
}

View File

@ -65,7 +65,7 @@ public class TestNumericRangeFilterBuilder extends LuceneTestCase {
try {
IndexReader reader = new SlowMultiReaderWrapper(IndexReader.open(ramDir, true));
try {
assertNull(filter.getDocIdSet((AtomicReaderContext) reader.getTopReaderContext()));
assertNull(filter.getDocIdSet((AtomicReaderContext) reader.getTopReaderContext(), reader.getLiveDocs()));
}
finally {
reader.close();

View File

@ -257,7 +257,7 @@ class PerSegmentSingleValuedFaceting {
// count collection array only needs to be as big as the number of terms we are
// going to collect counts for.
final int[] counts = this.counts = new int[nTerms];
DocIdSet idSet = baseSet.getDocIdSet(context);
DocIdSet idSet = baseSet.getDocIdSet(context, null); // this set only includes live docs
DocIdSetIterator iter = idSet.iterator();

View File

@ -17,10 +17,17 @@
package org.apache.solr.search;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.BitsFilteredDocIdSet;
import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.Filter;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.OpenBitSet;
import org.apache.lucene.util.OpenBitSetIterator;
import org.apache.lucene.search.DocIdSetIterator;
import java.io.IOException;
/**
* <code>BitDocSet</code> represents an unordered set of Lucene Document Ids
* using a BitSet. A set bit represents inclusion in the set for that document.
@ -231,4 +238,75 @@ public class BitDocSet extends DocSetBase {
protected BitDocSet clone() {
return new BitDocSet((OpenBitSet)bits.clone(), size);
}
@Override
public Filter getTopFilter() {
final OpenBitSet bs = bits;
// TODO: if cardinality isn't cached, do a quick measure of sparseness
// and return null from bits() if too sparse.
return new Filter() {
@Override
public DocIdSet getDocIdSet(final IndexReader.AtomicReaderContext context, final Bits acceptDocs) throws IOException {
IndexReader reader = context.reader;
if (context.isTopLevel) {
return BitsFilteredDocIdSet.wrap(bs, acceptDocs);
}
final int base = context.docBase;
final int maxDoc = reader.maxDoc();
final int max = base + maxDoc; // one past the max doc in this segment.
return BitsFilteredDocIdSet.wrap(new DocIdSet() {
@Override
public DocIdSetIterator iterator() throws IOException {
return new DocIdSetIterator() {
int pos=base-1;
int adjustedDoc=-1;
@Override
public int docID() {
return adjustedDoc;
}
@Override
public int nextDoc() throws IOException {
pos = bs.nextSetBit(pos+1);
return adjustedDoc = (pos>=0 && pos<max) ? pos-base : NO_MORE_DOCS;
}
@Override
public int advance(int target) throws IOException {
if (target==NO_MORE_DOCS) return adjustedDoc=NO_MORE_DOCS;
pos = bs.nextSetBit(target+base);
return adjustedDoc = (pos>=0 && pos<max) ? pos-base : NO_MORE_DOCS;
}
};
}
@Override
public boolean isCacheable() {
return true;
}
@Override
public Bits bits() throws IOException {
return new Bits() {
@Override
public boolean get(int index) {
return bs.fastGet(index + base);
}
@Override
public int length() {
return maxDoc;
}
};
}
}, acceptDocs);
}
};
}
}

View File

@ -18,10 +18,12 @@
package org.apache.solr.search;
import org.apache.solr.common.SolrException;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.OpenBitSet;
import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.BitsFilteredDocIdSet;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
@ -270,18 +272,18 @@ abstract class DocSetBase implements DocSet {
return new Filter() {
@Override
public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
public DocIdSet getDocIdSet(final AtomicReaderContext context, final Bits acceptDocs) throws IOException {
IndexReader reader = context.reader;
if (context.isTopLevel) {
return bs;
return BitsFilteredDocIdSet.wrap(bs, acceptDocs);
}
final int base = context.docBase;
final int maxDoc = reader.maxDoc();
final int max = base + maxDoc; // one past the max doc in this segment.
return new DocIdSet() {
return BitsFilteredDocIdSet.wrap(new DocIdSet() {
@Override
public DocIdSetIterator iterator() throws IOException {
return new DocIdSetIterator() {
@ -313,7 +315,13 @@ abstract class DocSetBase implements DocSet {
return true;
}
};
@Override
public Bits bits() throws IOException {
// sparse filters should not use random access
return null;
}
}, acceptDocs);
}
};
}

View File

@ -219,7 +219,7 @@ class JoinQuery extends Query {
filter = resultSet.getTopFilter();
}
DocIdSet readerSet = filter.getDocIdSet(context);
DocIdSet readerSet = filter.getDocIdSet(context, null); // this set only includes live docs
if (readerSet == null) readerSet=DocIdSet.EMPTY_DOCIDSET;
return new JoinScorer(this, readerSet.iterator(), getBoost());
}

View File

@ -121,13 +121,13 @@ public class SolrConstantScoreQuery extends ConstantScoreQuery implements Extend
@Override
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
boolean topScorer, Bits acceptDocs) throws IOException {
return new ConstantScorer(context, this, queryWeight);
return new ConstantScorer(context, this, queryWeight, acceptDocs);
}
@Override
public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
ConstantScorer cs = new ConstantScorer(context, this, queryWeight);
ConstantScorer cs = new ConstantScorer(context, this, queryWeight, context.reader.getLiveDocs());
boolean exists = cs.docIdSetIterator.advance(doc) == doc;
ComplexExplanation result = new ComplexExplanation();
@ -152,12 +152,14 @@ public class SolrConstantScoreQuery extends ConstantScoreQuery implements Extend
protected class ConstantScorer extends Scorer {
final DocIdSetIterator docIdSetIterator;
final float theScore;
final Bits acceptDocs;
int doc = -1;
public ConstantScorer(AtomicReaderContext context, ConstantWeight w, float theScore) throws IOException {
public ConstantScorer(AtomicReaderContext context, ConstantWeight w, float theScore, Bits acceptDocs) throws IOException {
super(w);
this.theScore = theScore;
DocIdSet docIdSet = filter instanceof SolrFilter ? ((SolrFilter)filter).getDocIdSet(w.context, context) : filter.getDocIdSet(context);
this.acceptDocs = acceptDocs;
DocIdSet docIdSet = filter instanceof SolrFilter ? ((SolrFilter)filter).getDocIdSet(w.context, context, acceptDocs) : filter.getDocIdSet(context, acceptDocs);
if (docIdSet == null) {
docIdSetIterator = DocIdSet.EMPTY_DOCIDSET.iterator();
} else {

View File

@ -20,6 +20,7 @@ package org.apache.solr.search;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.util.Bits;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
@ -39,10 +40,10 @@ public abstract class SolrFilter extends Filter {
* The context object will be passed to getDocIdSet() where this info can be retrieved. */
public abstract void createWeight(Map context, IndexSearcher searcher) throws IOException;
public abstract DocIdSet getDocIdSet(Map context, AtomicReaderContext readerContext) throws IOException;
public abstract DocIdSet getDocIdSet(Map context, AtomicReaderContext readerContext, Bits acceptDocs) throws IOException;
@Override
public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
return getDocIdSet(null, context);
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
return getDocIdSet(null, context, acceptDocs);
}
}

View File

@ -631,9 +631,10 @@ public class SolrIndexSearcher extends IndexSearcher implements SolrInfoMBean {
for (int i=0; i<leaves.length; i++) {
final AtomicReaderContext leaf = leaves[i];
final IndexReader reader = leaf.reader;
final Bits liveDocs = reader.getLiveDocs(); // TODO: the filter may already only have liveDocs...
DocIdSet idSet = null;
if (pf.filter != null) {
idSet = pf.filter.getDocIdSet(leaf);
idSet = pf.filter.getDocIdSet(leaf, liveDocs);
if (idSet == null) continue;
}
DocIdSetIterator idIter = null;
@ -643,7 +644,6 @@ public class SolrIndexSearcher extends IndexSearcher implements SolrInfoMBean {
}
collector.setNextReader(leaf);
Bits liveDocs = reader.getLiveDocs();
int max = reader.maxDoc();
if (idIter == null) {
@ -2056,8 +2056,8 @@ class FilterImpl extends Filter {
}
@Override
public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
DocIdSet sub = topFilter == null ? null : topFilter.getDocIdSet(context);
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
DocIdSet sub = topFilter == null ? null : topFilter.getDocIdSet(context, acceptDocs);
if (weights.size() == 0) return sub;
return new FilterSet(sub, context);
}
@ -2089,6 +2089,11 @@ class FilterImpl extends Filter {
if (iterators.size()==2) return new DualFilterIterator(iterators.get(0), iterators.get(1));
return new FilterIterator(iterators.toArray(new DocIdSetIterator[iterators.size()]));
}
@Override
public Bits bits() throws IOException {
return null; // don't use random access
}
}
private static class FilterIterator extends DocIdSetIterator {

View File

@ -17,7 +17,9 @@
package org.apache.solr.search;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.OpenBitSet;
import org.apache.lucene.search.BitsFilteredDocIdSet;
import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.Filter;
@ -655,7 +657,7 @@ public class SortedIntDocSet extends DocSetBase {
int lastEndIdx = 0;
@Override
public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
public DocIdSet getDocIdSet(final AtomicReaderContext context, final Bits acceptDocs) throws IOException {
IndexReader reader = context.reader;
final int base = context.docBase;
@ -688,7 +690,7 @@ public class SortedIntDocSet extends DocSetBase {
lastEndIdx = endIdx;
return new DocIdSet() {
return BitsFilteredDocIdSet.wrap(new DocIdSet() {
@Override
public DocIdSetIterator iterator() throws IOException {
return new DocIdSetIterator() {
@ -751,7 +753,13 @@ public class SortedIntDocSet extends DocSetBase {
return true;
}
};
@Override
public Bits bits() throws IOException {
// random access is expensive for this set
return null;
}
}, acceptDocs);
}
};
}

View File

@ -21,7 +21,9 @@ import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.BitsFilteredDocIdSet;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.util.Bits;
import org.apache.solr.search.SolrFilter;
import java.io.IOException;
@ -72,13 +74,17 @@ public class ValueSourceRangeFilter extends SolrFilter {
@Override
public DocIdSet getDocIdSet(final Map context, final AtomicReaderContext readerContext) throws IOException {
return new DocIdSet() {
public DocIdSet getDocIdSet(final Map context, final AtomicReaderContext readerContext, Bits acceptDocs) throws IOException {
return BitsFilteredDocIdSet.wrap(new DocIdSet() {
@Override
public DocIdSetIterator iterator() throws IOException {
public DocIdSetIterator iterator() throws IOException {
return valueSource.getValues(context, readerContext).getRangeScorer(readerContext.reader, lowerVal, upperVal, includeLower, includeUpper);
}
};
@Override
public Bits bits() throws IOException {
return null; // don't use random access
}
}, acceptDocs);
}
@Override

View File

@ -427,8 +427,8 @@ public class TestDocSet extends LuceneTestCase {
// first test in-sequence sub readers
for (AtomicReaderContext readerContext : ReaderUtil.leaves(topLevelContext)) {
da = fa.getDocIdSet(readerContext);
db = fb.getDocIdSet(readerContext);
da = fa.getDocIdSet(readerContext, null);
db = fb.getDocIdSet(readerContext, null);
doTestIteratorEqual(da, db);
}
@ -437,8 +437,8 @@ public class TestDocSet extends LuceneTestCase {
// now test out-of-sequence sub readers
for (int i=0; i<nReaders; i++) {
AtomicReaderContext readerContext = leaves[rand.nextInt(nReaders)];
da = fa.getDocIdSet(readerContext);
db = fb.getDocIdSet(readerContext);
da = fa.getDocIdSet(readerContext, null);
db = fb.getDocIdSet(readerContext, null);
doTestIteratorEqual(da, db);
}
}

View File

@ -31,6 +31,7 @@ import org.apache.lucene.search.*;
import org.apache.lucene.search.SortField.Type;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.OpenBitSet;
import org.apache.lucene.util._TestUtil;
import org.apache.solr.SolrTestCaseJ4;
@ -199,8 +200,8 @@ public class TestSort extends SolrTestCaseJ4 {
for (int i=0; i<qiter; i++) {
Filter filt = new Filter() {
@Override
public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
return randSet(context.reader.maxDoc());
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
return BitsFilteredDocIdSet.wrap(randSet(context.reader.maxDoc()), acceptDocs);
}
};