mirror of https://github.com/apache/lucene.git
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:
parent
a91efbedd1
commit
94f98ab7f3
|
@ -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
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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) {
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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];
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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++;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
|
@ -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+")";
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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; }
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
};
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
});
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
|
|
@ -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++;
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
||||
|
|
Loading…
Reference in New Issue