mirror of https://github.com/apache/lucene.git
LUCENE-6303: CachingWrapperFilter -> CachingWrapperQuery, FilterCache -> QueryCache and added caching to IndexSearcher.
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1662774 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
60ecddfb9e
commit
e4d88cab5c
|
@ -51,6 +51,9 @@ New Features
|
|||
* LUCENE-6294: Added oal.search.CollectorManager to allow for parallelization
|
||||
of the document collection process on IndexSearcher. (Adrien Grand)
|
||||
|
||||
* LUCENE-6303: Added filter caching baked into IndexSearcher and enabled by
|
||||
default. (Adrien Grand)
|
||||
|
||||
Bug Fixes
|
||||
|
||||
* LUCENE-6249: StandardQueryParser doesn't support pure negative clauses.
|
||||
|
@ -179,6 +182,9 @@ API Changes
|
|||
removed. Use PrefixQuery, TermRangeQuery and NumericRangeQuery instead.
|
||||
(Adrien Grand)
|
||||
|
||||
* LUCENE-6303: Replaced FilterCache with QueryCache and CachingWrapperFilter
|
||||
with CachingWrapperQuery. (Adrien Grand)
|
||||
|
||||
Other
|
||||
|
||||
* LUCENE-6248: Remove unused odd constants from StandardSyntaxParser.jj
|
||||
|
|
|
@ -20,7 +20,7 @@ package org.apache.lucene.index;
|
|||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
|
||||
import org.apache.lucene.search.CachingWrapperFilter;
|
||||
import org.apache.lucene.search.CachingWrapperQuery;
|
||||
import org.apache.lucene.util.AttributeSource;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
@ -38,7 +38,7 @@ import org.apache.lucene.util.BytesRef;
|
|||
* <p><b>NOTE</b>: If this {@link FilterLeafReader} does not change the
|
||||
* content the contained reader, you could consider overriding
|
||||
* {@link #getCoreCacheKey()} so that
|
||||
* {@link CachingWrapperFilter} shares the same entries for this atomic reader
|
||||
* {@link CachingWrapperQuery} shares the same entries for this atomic reader
|
||||
* and the wrapped one. {@link #getCombinedCoreAndDeletesKey()} could be
|
||||
* overridden as well if the {@link #getLiveDocs() live docs} are not changed
|
||||
* either.
|
||||
|
|
|
@ -25,6 +25,7 @@ import java.util.Set;
|
|||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.BooleanClause.Occur;
|
||||
import org.apache.lucene.search.similarities.Similarity;
|
||||
import org.apache.lucene.util.ToStringUtils;
|
||||
|
||||
|
@ -158,9 +159,27 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
|
|||
@Override
|
||||
public final Iterator<BooleanClause> iterator() { return clauses().iterator(); }
|
||||
|
||||
private static BooleanQuery downgradeMustClauseToFilter(BooleanQuery bq) {
|
||||
BooleanQuery clone = bq.clone();
|
||||
clone.clauses.clear();
|
||||
for (BooleanClause clause : bq.clauses()) {
|
||||
if (clause.getOccur() == Occur.MUST) {
|
||||
clone.add(clause.getQuery(), Occur.FILTER);
|
||||
} else {
|
||||
clone.add(clause);
|
||||
}
|
||||
}
|
||||
return clone;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||
return new BooleanWeight(this, searcher, needsScores, disableCoord);
|
||||
BooleanQuery query = this;
|
||||
if (needsScores == false) {
|
||||
// we rewrite MUST clauses to FILTER for caching
|
||||
query = downgradeMustClauseToFilter(query);
|
||||
}
|
||||
return new BooleanWeight(query, searcher, needsScores, disableCoord);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -52,7 +52,7 @@ public class BooleanWeight extends Weight {
|
|||
weights = new ArrayList<>(query.clauses().size());
|
||||
for (int i = 0 ; i < query.clauses().size(); i++) {
|
||||
BooleanClause c = query.clauses().get(i);
|
||||
Weight w = c.getQuery().createWeight(searcher, needsScores && c.isScoring());
|
||||
Weight w = searcher.createWeight(c.getQuery(), needsScores && c.isScoring());
|
||||
weights.add(w);
|
||||
if (c.isScoring()) {
|
||||
maxCoord++;
|
||||
|
|
|
@ -1,170 +0,0 @@
|
|||
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 static org.apache.lucene.search.DocIdSet.EMPTY;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.WeakHashMap;
|
||||
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.Accountables;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.RoaringDocIdSet;
|
||||
|
||||
/**
|
||||
* Wraps another {@link Filter}'s result and caches it. The purpose is to allow
|
||||
* filters to simply filter, and then wrap with this class
|
||||
* to add caching.
|
||||
*/
|
||||
public class CachingWrapperFilter extends Filter implements Accountable {
|
||||
private final Filter filter;
|
||||
private final FilterCachingPolicy policy;
|
||||
private final Map<Object,DocIdSet> cache = Collections.synchronizedMap(new WeakHashMap<Object,DocIdSet>());
|
||||
|
||||
/** Wraps another filter's result and caches it according to the provided policy.
|
||||
* @param filter Filter to cache results of
|
||||
* @param policy policy defining which filters should be cached on which segments
|
||||
*/
|
||||
public CachingWrapperFilter(Filter filter, FilterCachingPolicy policy) {
|
||||
this.filter = filter;
|
||||
this.policy = policy;
|
||||
}
|
||||
|
||||
/** Same as {@link CachingWrapperFilter#CachingWrapperFilter(Filter, FilterCachingPolicy)}
|
||||
* but enforces the use of the
|
||||
* {@link FilterCachingPolicy.CacheOnLargeSegments#DEFAULT} policy. */
|
||||
public CachingWrapperFilter(Filter filter) {
|
||||
this(filter, FilterCachingPolicy.CacheOnLargeSegments.DEFAULT);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the contained filter.
|
||||
* @return the contained filter.
|
||||
*/
|
||||
public Filter getFilter() {
|
||||
return filter;
|
||||
}
|
||||
|
||||
/**
|
||||
* 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 calls
|
||||
* {@link #cacheImpl(DocIdSetIterator, org.apache.lucene.index.LeafReader)}
|
||||
* <p>Note: This method returns {@linkplain DocIdSet#EMPTY} if the given docIdSet
|
||||
* is <code>null</code> or if {@link DocIdSet#iterator()} return <code>null</code>. The empty
|
||||
* instance is use as a placeholder in the cache instead of the <code>null</code> value.
|
||||
*/
|
||||
protected DocIdSet docIdSetToCache(DocIdSet docIdSet, LeafReader reader) throws IOException {
|
||||
if (docIdSet == null || docIdSet.isCacheable()) {
|
||||
return docIdSet;
|
||||
} else {
|
||||
final DocIdSetIterator it = docIdSet.iterator();
|
||||
if (it == null) {
|
||||
return null;
|
||||
} else {
|
||||
return cacheImpl(it, reader);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Default cache implementation: uses {@link RoaringDocIdSet}.
|
||||
*/
|
||||
protected DocIdSet cacheImpl(DocIdSetIterator iterator, LeafReader reader) throws IOException {
|
||||
return new RoaringDocIdSet.Builder(reader.maxDoc()).add(iterator).build();
|
||||
}
|
||||
|
||||
// for testing
|
||||
int hitCount, missCount;
|
||||
|
||||
@Override
|
||||
public DocIdSet getDocIdSet(LeafReaderContext context, final Bits acceptDocs) throws IOException {
|
||||
final LeafReader reader = context.reader();
|
||||
final Object key = reader.getCoreCacheKey();
|
||||
|
||||
DocIdSet docIdSet = cache.get(key);
|
||||
if (docIdSet != null) {
|
||||
hitCount++;
|
||||
} else {
|
||||
docIdSet = filter.getDocIdSet(context, null);
|
||||
if (policy.shouldCache(filter, context, docIdSet)) {
|
||||
missCount++;
|
||||
docIdSet = docIdSetToCache(docIdSet, reader);
|
||||
if (docIdSet == null) {
|
||||
// We use EMPTY as a sentinel for the empty set, which is cacheable
|
||||
docIdSet = EMPTY;
|
||||
}
|
||||
assert docIdSet.isCacheable();
|
||||
cache.put(key, docIdSet);
|
||||
}
|
||||
}
|
||||
|
||||
return docIdSet == EMPTY ? null : BitsFilteredDocIdSet.wrap(docIdSet, acceptDocs);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString(String field) {
|
||||
return getClass().getSimpleName() + "("+filter.toString(field)+")";
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (o == null || !getClass().equals(o.getClass())) return false;
|
||||
final CachingWrapperFilter other = (CachingWrapperFilter) o;
|
||||
return this.filter.equals(other.filter);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return (filter.hashCode() ^ getClass().hashCode());
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
|
||||
// Sync only to pull the current set of values:
|
||||
List<DocIdSet> docIdSets;
|
||||
synchronized(cache) {
|
||||
docIdSets = new ArrayList<>(cache.values());
|
||||
}
|
||||
|
||||
long total = 0;
|
||||
for(DocIdSet dis : docIdSets) {
|
||||
total += dis.ramBytesUsed();
|
||||
}
|
||||
|
||||
return total;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<Accountable> getChildResources() {
|
||||
// Sync to pull the current set of values:
|
||||
synchronized (cache) {
|
||||
// no need to clone, Accountable#namedAccountables already copies the data
|
||||
return Accountables.namedAccountables("segment", cache);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,253 @@
|
|||
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 java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.WeakHashMap;
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.Accountables;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.RoaringDocIdSet;
|
||||
|
||||
/**
|
||||
* Wraps another {@link Query}'s result and caches it when scores are not
|
||||
* needed. The purpose is to allow queries to simply care about matching and
|
||||
* scoring, and then wrap with this class to add caching.
|
||||
*/
|
||||
public class CachingWrapperQuery extends Query implements Accountable {
|
||||
private Query query; // not final because of clone
|
||||
private final QueryCachingPolicy policy;
|
||||
private final Map<Object,DocIdSet> cache = Collections.synchronizedMap(new WeakHashMap<Object,DocIdSet>());
|
||||
|
||||
/** Wraps another query's result and caches it according to the provided policy.
|
||||
* @param query Query to cache results of
|
||||
* @param policy policy defining which filters should be cached on which segments
|
||||
*/
|
||||
public CachingWrapperQuery(Query query, QueryCachingPolicy policy) {
|
||||
this.query = query;
|
||||
this.policy = policy;
|
||||
}
|
||||
|
||||
/** Same as {@link CachingWrapperQuery#CachingWrapperQuery(Query, QueryCachingPolicy)}
|
||||
* but enforces the use of the
|
||||
* {@link QueryCachingPolicy.CacheOnLargeSegments#DEFAULT} policy. */
|
||||
public CachingWrapperQuery(Query query) {
|
||||
this(query, QueryCachingPolicy.CacheOnLargeSegments.DEFAULT);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CachingWrapperQuery clone() {
|
||||
final CachingWrapperQuery clone = (CachingWrapperQuery) super.clone();
|
||||
clone.query = query.clone();
|
||||
return clone;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the contained query.
|
||||
* @return the contained query.
|
||||
*/
|
||||
public Query getQuery() {
|
||||
return query;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getBoost() {
|
||||
return query.getBoost();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setBoost(float b) {
|
||||
query.setBoost(b);
|
||||
}
|
||||
|
||||
/**
|
||||
* Default cache implementation: uses {@link RoaringDocIdSet}.
|
||||
*/
|
||||
protected DocIdSet cacheImpl(DocIdSetIterator iterator, LeafReader reader) throws IOException {
|
||||
return new RoaringDocIdSet.Builder(reader.maxDoc()).add(iterator).build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Query rewrite(IndexReader reader) throws IOException {
|
||||
final Query rewritten = query.rewrite(reader);
|
||||
if (query == rewritten) {
|
||||
return this;
|
||||
} else {
|
||||
CachingWrapperQuery clone = clone();
|
||||
clone.query = rewritten;
|
||||
return clone;
|
||||
}
|
||||
}
|
||||
|
||||
// for testing
|
||||
int hitCount, missCount;
|
||||
|
||||
@Override
|
||||
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||
final Weight weight = query.createWeight(searcher, needsScores);
|
||||
if (needsScores) {
|
||||
// our cache is not sufficient, we need scores too
|
||||
return weight;
|
||||
}
|
||||
policy.onUse(weight.getQuery());
|
||||
return new ConstantScoreWeight(weight.getQuery()) {
|
||||
@Override
|
||||
Scorer scorer(LeafReaderContext context, Bits acceptDocs, float score) throws IOException {
|
||||
final LeafReader reader = context.reader();
|
||||
final Object key = reader.getCoreCacheKey();
|
||||
|
||||
DocIdSet docIdSet = cache.get(key);
|
||||
if (docIdSet != null) {
|
||||
hitCount++;
|
||||
} else if (policy.shouldCache(query, context)) {
|
||||
missCount++;
|
||||
final Scorer scorer = weight.scorer(context, null);
|
||||
if (scorer == null) {
|
||||
docIdSet = DocIdSet.EMPTY;
|
||||
} else {
|
||||
docIdSet = cacheImpl(scorer, context.reader());
|
||||
}
|
||||
cache.put(key, docIdSet);
|
||||
} else {
|
||||
return weight.scorer(context, acceptDocs);
|
||||
}
|
||||
|
||||
assert docIdSet != null;
|
||||
if (docIdSet == DocIdSet.EMPTY) {
|
||||
return null;
|
||||
}
|
||||
final DocIdSetIterator approximation = docIdSet.iterator();
|
||||
if (approximation == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
final DocIdSetIterator disi;
|
||||
final TwoPhaseIterator twoPhaseView;
|
||||
if (acceptDocs == null) {
|
||||
twoPhaseView = null;
|
||||
disi = approximation;
|
||||
} else {
|
||||
twoPhaseView = new TwoPhaseIterator() {
|
||||
|
||||
@Override
|
||||
public boolean matches() throws IOException {
|
||||
final int doc = approximation.docID();
|
||||
return acceptDocs.get(doc);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocIdSetIterator approximation() {
|
||||
return approximation;
|
||||
}
|
||||
};
|
||||
disi = TwoPhaseIterator.asDocIdSetIterator(twoPhaseView);
|
||||
}
|
||||
return new Scorer(weight) {
|
||||
|
||||
@Override
|
||||
public TwoPhaseIterator asTwoPhaseIterator() {
|
||||
return twoPhaseView;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float score() throws IOException {
|
||||
return 0f;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int freq() throws IOException {
|
||||
return 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return disi.docID();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() throws IOException {
|
||||
return disi.nextDoc();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
return disi.advance(target);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long cost() {
|
||||
return disi.cost();
|
||||
}
|
||||
|
||||
};
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString(String field) {
|
||||
return getClass().getSimpleName() + "("+query.toString(field)+")";
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (o == null || !getClass().equals(o.getClass())) return false;
|
||||
final CachingWrapperQuery other = (CachingWrapperQuery) o;
|
||||
return this.query.equals(other.query);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return (query.hashCode() ^ getClass().hashCode());
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
|
||||
// Sync only to pull the current set of values:
|
||||
List<DocIdSet> docIdSets;
|
||||
synchronized(cache) {
|
||||
docIdSets = new ArrayList<>(cache.values());
|
||||
}
|
||||
|
||||
long total = 0;
|
||||
for(DocIdSet dis : docIdSets) {
|
||||
total += dis.ramBytesUsed();
|
||||
}
|
||||
|
||||
return total;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<Accountable> getChildResources() {
|
||||
// Sync to pull the current set of values:
|
||||
synchronized (cache) {
|
||||
// no need to clone, Accountable#namedAccountables already copies the data
|
||||
return Accountables.namedAccountables("segment", cache);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -256,7 +256,7 @@ public class ConstantScoreQuery extends Query {
|
|||
|
||||
@Override
|
||||
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||
final Weight innerWeight = query.createWeight(searcher, false);
|
||||
final Weight innerWeight = searcher.createWeight(query, false);
|
||||
if (needsScores) {
|
||||
return new ConstantScoreQuery.ConstantWeight(innerWeight);
|
||||
} else {
|
||||
|
|
|
@ -36,11 +36,6 @@ public abstract class DocIdSet implements Accountable {
|
|||
return DocIdSetIterator.empty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isCacheable() {
|
||||
return true;
|
||||
}
|
||||
|
||||
// we explicitly provide no random access, as this filter is 100% sparse and iterator exits faster
|
||||
@Override
|
||||
public Bits bits() {
|
||||
|
@ -82,14 +77,4 @@ public abstract class DocIdSet implements Accountable {
|
|||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* This method is a hint for {@link CachingWrapperFilter}, if this <code>DocIdSet</code>
|
||||
* should be cached without copying it. The default is to return
|
||||
* <code>false</code>. If you have an own <code>DocIdSet</code> implementation
|
||||
* that does its iteration very effective and fast without doing disk I/O,
|
||||
* override this method and return <code>true</code>.
|
||||
*/
|
||||
public boolean isCacheable() {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -58,12 +58,6 @@ public abstract class FilteredDocIdSet extends DocIdSet {
|
|||
return _innerSet;
|
||||
}
|
||||
|
||||
/** This DocIdSet implementation is cacheable if the inner set is cacheable. */
|
||||
@Override
|
||||
public boolean isCacheable() {
|
||||
return _innerSet.isCacheable();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return RamUsageEstimator.NUM_BYTES_OBJECT_REF + _innerSet.ramBytesUsed();
|
||||
|
|
|
@ -37,7 +37,7 @@ import org.apache.lucene.util.ToStringUtils;
|
|||
* query is used in a search - use a CachingWrapperFilter to avoid
|
||||
* regenerating the bits every time.
|
||||
* @since 1.4
|
||||
* @see CachingWrapperFilter
|
||||
* @see CachingWrapperQuery
|
||||
*/
|
||||
public class FilteredQuery extends Query {
|
||||
|
||||
|
|
|
@ -23,6 +23,7 @@ import java.util.Arrays;
|
|||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
|
@ -72,6 +73,11 @@ import org.apache.lucene.util.ThreadInterruptedException;
|
|||
* use your own (non-Lucene) objects instead.</p>
|
||||
*/
|
||||
public class IndexSearcher {
|
||||
|
||||
// 32MB and at most 10,000 queries
|
||||
private static final QueryCache DEFAULT_QUERY_CACHE = new LRUQueryCache(10000, 1 << 25);
|
||||
private static final QueryCachingPolicy DEFAULT_CACHING_POLICY = new UsageTrackingQueryCachingPolicy();
|
||||
|
||||
final IndexReader reader; // package private for testing!
|
||||
|
||||
// NOTE: these members might change in incompatible ways
|
||||
|
@ -86,7 +92,10 @@ public class IndexSearcher {
|
|||
|
||||
// the default Similarity
|
||||
private static final Similarity defaultSimilarity = new DefaultSimilarity();
|
||||
|
||||
|
||||
private QueryCache queryCache = DEFAULT_QUERY_CACHE;
|
||||
private QueryCachingPolicy queryCachingPolicy = DEFAULT_CACHING_POLICY;
|
||||
|
||||
/**
|
||||
* Expert: returns a default Similarity instance.
|
||||
* In general, this method is only called to initialize searchers and writers.
|
||||
|
@ -156,7 +165,28 @@ public class IndexSearcher {
|
|||
public IndexSearcher(IndexReaderContext context) {
|
||||
this(context, null);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Set the {@link QueryCache} to use when scores are not needed.
|
||||
* A value of {@code null} indicates that query matches should never be
|
||||
* cached. This method should be called <b>before</b> starting using this
|
||||
* {@link IndexSearcher}.
|
||||
* @see QueryCache
|
||||
*/
|
||||
public void setQueryCache(QueryCache queryCache) {
|
||||
this.queryCache = queryCache;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the {@link QueryCachingPolicy} to use for query caching.
|
||||
* This method should be called <b>before</b> starting using this
|
||||
* {@link IndexSearcher}.
|
||||
* @see QueryCachingPolicy
|
||||
*/
|
||||
public void setQueryCachingPolicy(QueryCachingPolicy queryCachingPolicy) {
|
||||
this.queryCachingPolicy = Objects.requireNonNull(queryCachingPolicy);
|
||||
}
|
||||
|
||||
/**
|
||||
* Expert: Creates an array of leaf slices each holding a subset of the given leaves.
|
||||
* Each {@link LeafSlice} is executed in a single thread. By default there
|
||||
|
@ -557,7 +587,7 @@ public class IndexSearcher {
|
|||
*/
|
||||
public Weight createNormalizedWeight(Query query, boolean needsScores) throws IOException {
|
||||
query = rewrite(query);
|
||||
Weight weight = query.createWeight(this, needsScores);
|
||||
Weight weight = createWeight(query, needsScores);
|
||||
float v = weight.getValueForNormalization();
|
||||
float norm = getSimilarity().queryNorm(v);
|
||||
if (Float.isInfinite(norm) || Float.isNaN(norm)) {
|
||||
|
@ -566,7 +596,21 @@ public class IndexSearcher {
|
|||
weight.normalize(norm, 1.0f);
|
||||
return weight;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Creates a {@link Weight} for the given query, potentially adding caching
|
||||
* if possible and configured.
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public Weight createWeight(Query query, boolean needsScores) throws IOException {
|
||||
final QueryCache queryCache = this.queryCache;
|
||||
Weight weight = query.createWeight(this, needsScores);
|
||||
if (needsScores == false && queryCache != null) {
|
||||
weight = queryCache.doCache(weight, queryCachingPolicy);
|
||||
}
|
||||
return weight;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns this searchers the top-level {@link IndexReaderContext}.
|
||||
* @see IndexReader#getContext()
|
||||
|
|
|
@ -38,34 +38,34 @@ import org.apache.lucene.util.RamUsageEstimator;
|
|||
import org.apache.lucene.util.RoaringDocIdSet;
|
||||
|
||||
/**
|
||||
* A {@link FilterCache} that evicts filters using a LRU (least-recently-used)
|
||||
* A {@link QueryCache} that evicts queries using a LRU (least-recently-used)
|
||||
* eviction policy in order to remain under a given maximum size and number of
|
||||
* bytes used.
|
||||
*
|
||||
* This class is thread-safe.
|
||||
*
|
||||
* Note that filter eviction runs in linear time with the total number of
|
||||
* Note that query eviction runs in linear time with the total number of
|
||||
* segments that have cache entries so this cache works best with
|
||||
* {@link FilterCachingPolicy caching policies} that only cache on "large"
|
||||
* {@link QueryCachingPolicy caching policies} that only cache on "large"
|
||||
* segments, and it is advised to not share this cache across too many indices.
|
||||
*
|
||||
* Typical usage looks like this:
|
||||
* <pre class="prettyprint">
|
||||
* final int maxNumberOfCachedFilters = 256;
|
||||
* final int maxNumberOfCachedQueries = 256;
|
||||
* final long maxRamBytesUsed = 50 * 1024L * 1024L; // 50MB
|
||||
* // these cache and policy instances can be shared across several filters and readers
|
||||
* // these cache and policy instances can be shared across several queries and readers
|
||||
* // it is fine to eg. store them into static variables
|
||||
* final FilterCache filterCache = new LRUFilterCache(maxNumberOfCachedFilters, maxRamBytesUsed);
|
||||
* final FilterCachingPolicy defaultCachingPolicy = new UsageTrackingFilterCachingPolicy();
|
||||
*
|
||||
* final QueryCache queryCache = new LRUQueryCache(maxNumberOfCachedQueries, maxRamBytesUsed);
|
||||
* final QueryCachingPolicy defaultCachingPolicy = new UsageTrackingQueryCachingPolicy();
|
||||
*
|
||||
* // ...
|
||||
*
|
||||
*
|
||||
* // Then at search time
|
||||
* Filter myFilter = ...;
|
||||
* Filter myCacheFilter = filterCache.doCache(myFilter, defaultCachingPolicy);
|
||||
* // myCacheFilter is now a wrapper around the original filter that will interact with the cache
|
||||
* Query myQuery = ...;
|
||||
* Query myCacheQuery = queryCache.doCache(myQuery, defaultCachingPolicy);
|
||||
* // myCacheQuery is now a wrapper around the original query that will interact with the cache
|
||||
* IndexSearcher searcher = ...;
|
||||
* TopDocs topDocs = searcher.search(new ConstantScoreQuery(myCacheFilter), 10);
|
||||
* TopDocs topDocs = searcher.search(new ConstantScoreQuery(myCacheQuery), 10);
|
||||
* </pre>
|
||||
*
|
||||
* This cache exposes some global statistics ({@link #getHitCount() hit count},
|
||||
|
@ -73,20 +73,20 @@ import org.apache.lucene.util.RoaringDocIdSet;
|
|||
* entries}, {@link #getCacheCount() total number of DocIdSets that have ever
|
||||
* been cached}, {@link #getEvictionCount() number of evicted entries}). In
|
||||
* case you would like to have more fine-grained statistics, such as per-index
|
||||
* or per-filter-class statistics, it is possible to override various callbacks:
|
||||
* or per-query-class statistics, it is possible to override various callbacks:
|
||||
* {@link #onHit}, {@link #onMiss},
|
||||
* {@link #onFilterCache}, {@link #onFilterEviction},
|
||||
* {@link #onQueryCache}, {@link #onQueryEviction},
|
||||
* {@link #onDocIdSetCache}, {@link #onDocIdSetEviction} and {@link #onClear}.
|
||||
* It is better to not perform heavy computations in these methods though since
|
||||
* they are called synchronously and under a lock.
|
||||
*
|
||||
* @see FilterCachingPolicy
|
||||
* @see QueryCachingPolicy
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class LRUFilterCache implements FilterCache, Accountable {
|
||||
public class LRUQueryCache implements QueryCache, Accountable {
|
||||
|
||||
// memory usage of a simple query-wrapper filter around a term query
|
||||
static final long FILTER_DEFAULT_RAM_BYTES_USED = 216;
|
||||
// memory usage of a simple term query
|
||||
static final long QUERY_DEFAULT_RAM_BYTES_USED = 192;
|
||||
|
||||
static final long HASHTABLE_RAM_BYTES_PER_ENTRY =
|
||||
2 * RamUsageEstimator.NUM_BYTES_OBJECT_REF // key + value
|
||||
|
@ -98,13 +98,13 @@ public class LRUFilterCache implements FilterCache, Accountable {
|
|||
|
||||
private final int maxSize;
|
||||
private final long maxRamBytesUsed;
|
||||
// maps filters that are contained in the cache to a singleton so that this
|
||||
// cache does not store several copies of the same filter
|
||||
private final Map<Filter, Filter> uniqueFilters;
|
||||
// maps queries that are contained in the cache to a singleton so that this
|
||||
// cache does not store several copies of the same query
|
||||
private final Map<Query, Query> uniqueQueries;
|
||||
// The contract between this set and the per-leaf caches is that per-leaf caches
|
||||
// are only allowed to store sub-sets of the filters that are contained in
|
||||
// mostRecentlyUsedFilters. This is why write operations are performed under a lock
|
||||
private final Set<Filter> mostRecentlyUsedFilters;
|
||||
// are only allowed to store sub-sets of the queries that are contained in
|
||||
// mostRecentlyUsedQueries. This is why write operations are performed under a lock
|
||||
private final Set<Query> mostRecentlyUsedQueries;
|
||||
private final Map<Object, LeafCache> cache;
|
||||
|
||||
// these variables are volatile so that we do not need to sync reads
|
||||
|
@ -116,63 +116,63 @@ public class LRUFilterCache implements FilterCache, Accountable {
|
|||
private volatile long cacheSize;
|
||||
|
||||
/**
|
||||
* Create a new instance that will cache at most <code>maxSize</code> filters
|
||||
* Create a new instance that will cache at most <code>maxSize</code> queries
|
||||
* with at most <code>maxRamBytesUsed</code> bytes of memory.
|
||||
*/
|
||||
public LRUFilterCache(int maxSize, long maxRamBytesUsed) {
|
||||
public LRUQueryCache(int maxSize, long maxRamBytesUsed) {
|
||||
this.maxSize = maxSize;
|
||||
this.maxRamBytesUsed = maxRamBytesUsed;
|
||||
uniqueFilters = new LinkedHashMap<Filter, Filter>(16, 0.75f, true);
|
||||
mostRecentlyUsedFilters = uniqueFilters.keySet();
|
||||
uniqueQueries = new LinkedHashMap<>(16, 0.75f, true);
|
||||
mostRecentlyUsedQueries = uniqueQueries.keySet();
|
||||
cache = new IdentityHashMap<>();
|
||||
ramBytesUsed = 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* Expert: callback when there is a cache hit on a given filter.
|
||||
* Expert: callback when there is a cache hit on a given query.
|
||||
* Implementing this method is typically useful in order to compute more
|
||||
* fine-grained statistics about the filter cache.
|
||||
* fine-grained statistics about the query cache.
|
||||
* @see #onMiss
|
||||
* @lucene.experimental
|
||||
*/
|
||||
protected void onHit(Object readerCoreKey, Filter filter) {
|
||||
protected void onHit(Object readerCoreKey, Query query) {
|
||||
hitCount += 1;
|
||||
}
|
||||
|
||||
/**
|
||||
* Expert: callback when there is a cache miss on a given filter.
|
||||
* Expert: callback when there is a cache miss on a given query.
|
||||
* @see #onHit
|
||||
* @lucene.experimental
|
||||
*/
|
||||
protected void onMiss(Object readerCoreKey, Filter filter) {
|
||||
assert filter != null;
|
||||
protected void onMiss(Object readerCoreKey, Query query) {
|
||||
assert query != null;
|
||||
missCount += 1;
|
||||
}
|
||||
|
||||
/**
|
||||
* Expert: callback when a filter is added to this cache.
|
||||
* Expert: callback when a query is added to this cache.
|
||||
* Implementing this method is typically useful in order to compute more
|
||||
* fine-grained statistics about the filter cache.
|
||||
* @see #onFilterEviction
|
||||
* fine-grained statistics about the query cache.
|
||||
* @see #onQueryEviction
|
||||
* @lucene.experimental
|
||||
*/
|
||||
protected void onFilterCache(Filter filter, long ramBytesUsed) {
|
||||
protected void onQueryCache(Query query, long ramBytesUsed) {
|
||||
this.ramBytesUsed += ramBytesUsed;
|
||||
}
|
||||
|
||||
/**
|
||||
* Expert: callback when a filter is evicted from this cache.
|
||||
* @see #onFilterCache
|
||||
* Expert: callback when a query is evicted from this cache.
|
||||
* @see #onQueryCache
|
||||
* @lucene.experimental
|
||||
*/
|
||||
protected void onFilterEviction(Filter filter, long ramBytesUsed) {
|
||||
protected void onQueryEviction(Query query, long ramBytesUsed) {
|
||||
this.ramBytesUsed -= ramBytesUsed;
|
||||
}
|
||||
|
||||
/**
|
||||
* Expert: callback when a {@link DocIdSet} is added to this cache.
|
||||
* Implementing this method is typically useful in order to compute more
|
||||
* fine-grained statistics about the filter cache.
|
||||
* fine-grained statistics about the query cache.
|
||||
* @see #onDocIdSetEviction
|
||||
* @lucene.experimental
|
||||
*/
|
||||
|
@ -181,7 +181,7 @@ public class LRUFilterCache implements FilterCache, Accountable {
|
|||
cacheCount += 1;
|
||||
this.ramBytesUsed += ramBytesUsed;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Expert: callback when one or more {@link DocIdSet}s are removed from this
|
||||
* cache.
|
||||
|
@ -204,7 +204,7 @@ public class LRUFilterCache implements FilterCache, Accountable {
|
|||
|
||||
/** Whether evictions are required. */
|
||||
boolean requiresEviction() {
|
||||
final int size = mostRecentlyUsedFilters.size();
|
||||
final int size = mostRecentlyUsedQueries.size();
|
||||
if (size == 0) {
|
||||
return false;
|
||||
} else {
|
||||
|
@ -212,17 +212,18 @@ public class LRUFilterCache implements FilterCache, Accountable {
|
|||
}
|
||||
}
|
||||
|
||||
synchronized DocIdSet get(Filter filter, LeafReaderContext context) {
|
||||
synchronized DocIdSet get(Query key, LeafReaderContext context) {
|
||||
key = QueryCache.cacheKey(key);
|
||||
final Object readerKey = context.reader().getCoreCacheKey();
|
||||
final LeafCache leafCache = cache.get(readerKey);
|
||||
if (leafCache == null) {
|
||||
onMiss(readerKey, filter);
|
||||
onMiss(readerKey, key);
|
||||
return null;
|
||||
}
|
||||
// this get call moves the filter to the most-recently-used position
|
||||
final Filter singleton = uniqueFilters.get(filter);
|
||||
// this get call moves the query to the most-recently-used position
|
||||
final Query singleton = uniqueQueries.get(key);
|
||||
if (singleton == null) {
|
||||
onMiss(readerKey, filter);
|
||||
onMiss(readerKey, key);
|
||||
return null;
|
||||
}
|
||||
final DocIdSet cached = leafCache.get(singleton);
|
||||
|
@ -234,14 +235,17 @@ public class LRUFilterCache implements FilterCache, Accountable {
|
|||
return cached;
|
||||
}
|
||||
|
||||
synchronized void putIfAbsent(Filter filter, LeafReaderContext context, DocIdSet set) {
|
||||
// under a lock to make sure that mostRecentlyUsedFilters and cache remain sync'ed
|
||||
assert set.isCacheable();
|
||||
Filter singleton = uniqueFilters.putIfAbsent(filter, filter);
|
||||
synchronized void putIfAbsent(Query query, LeafReaderContext context, DocIdSet set) {
|
||||
// under a lock to make sure that mostRecentlyUsedQueries and cache remain sync'ed
|
||||
// we don't want to have user-provided queries as keys in our cache since queries are mutable
|
||||
query = query.clone();
|
||||
query.setBoost(1f);
|
||||
assert query == QueryCache.cacheKey(query);
|
||||
Query singleton = uniqueQueries.putIfAbsent(query, query);
|
||||
if (singleton == null) {
|
||||
onFilterCache(singleton, LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY + ramBytesUsed(filter));
|
||||
onQueryCache(singleton, LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY + ramBytesUsed(query));
|
||||
} else {
|
||||
filter = singleton;
|
||||
query = singleton;
|
||||
}
|
||||
final Object key = context.reader().getCoreCacheKey();
|
||||
LeafCache leafCache = cache.get(key);
|
||||
|
@ -258,18 +262,18 @@ public class LRUFilterCache implements FilterCache, Accountable {
|
|||
}
|
||||
});
|
||||
}
|
||||
leafCache.putIfAbsent(filter, set);
|
||||
leafCache.putIfAbsent(query, set);
|
||||
evictIfNecessary();
|
||||
}
|
||||
|
||||
synchronized void evictIfNecessary() {
|
||||
// under a lock to make sure that mostRecentlyUsedFilters and cache keep sync'ed
|
||||
// under a lock to make sure that mostRecentlyUsedQueries and cache keep sync'ed
|
||||
if (requiresEviction()) {
|
||||
Iterator<Filter> iterator = mostRecentlyUsedFilters.iterator();
|
||||
Iterator<Query> iterator = mostRecentlyUsedQueries.iterator();
|
||||
do {
|
||||
final Filter filter = iterator.next();
|
||||
final Query query = iterator.next();
|
||||
iterator.remove();
|
||||
onEviction(filter);
|
||||
onEviction(query);
|
||||
} while (iterator.hasNext() && requiresEviction());
|
||||
}
|
||||
}
|
||||
|
@ -286,17 +290,17 @@ public class LRUFilterCache implements FilterCache, Accountable {
|
|||
}
|
||||
|
||||
/**
|
||||
* Remove all cache entries for the given filter.
|
||||
* Remove all cache entries for the given query.
|
||||
*/
|
||||
public synchronized void clearFilter(Filter filter) {
|
||||
final Filter singleton = uniqueFilters.remove(filter);
|
||||
public synchronized void clearQuery(Query query) {
|
||||
final Query singleton = uniqueQueries.remove(QueryCache.cacheKey(query));
|
||||
if (singleton != null) {
|
||||
onEviction(singleton);
|
||||
}
|
||||
}
|
||||
|
||||
private void onEviction(Filter singleton) {
|
||||
onFilterEviction(singleton, LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY + ramBytesUsed(singleton));
|
||||
private void onEviction(Query singleton) {
|
||||
onQueryEviction(singleton, LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY + ramBytesUsed(singleton));
|
||||
for (LeafCache leafCache : cache.values()) {
|
||||
leafCache.remove(singleton);
|
||||
}
|
||||
|
@ -307,29 +311,29 @@ public class LRUFilterCache implements FilterCache, Accountable {
|
|||
*/
|
||||
public synchronized void clear() {
|
||||
cache.clear();
|
||||
mostRecentlyUsedFilters.clear();
|
||||
mostRecentlyUsedQueries.clear();
|
||||
onClear();
|
||||
}
|
||||
|
||||
// pkg-private for testing
|
||||
synchronized void assertConsistent() {
|
||||
if (requiresEviction()) {
|
||||
throw new AssertionError("requires evictions: size=" + mostRecentlyUsedFilters.size()
|
||||
throw new AssertionError("requires evictions: size=" + mostRecentlyUsedQueries.size()
|
||||
+ ", maxSize=" + maxSize + ", ramBytesUsed=" + ramBytesUsed() + ", maxRamBytesUsed=" + maxRamBytesUsed);
|
||||
}
|
||||
for (LeafCache leafCache : cache.values()) {
|
||||
Set<Filter> keys = Collections.newSetFromMap(new IdentityHashMap<>());
|
||||
Set<Query> keys = Collections.newSetFromMap(new IdentityHashMap<>());
|
||||
keys.addAll(leafCache.cache.keySet());
|
||||
keys.removeAll(mostRecentlyUsedFilters);
|
||||
keys.removeAll(mostRecentlyUsedQueries);
|
||||
if (!keys.isEmpty()) {
|
||||
throw new AssertionError("One leaf cache contains more keys than the top-level cache: " + keys);
|
||||
}
|
||||
}
|
||||
long recomputedRamBytesUsed =
|
||||
HASHTABLE_RAM_BYTES_PER_ENTRY * cache.size()
|
||||
+ LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY * uniqueFilters.size();
|
||||
for (Filter filter : mostRecentlyUsedFilters) {
|
||||
recomputedRamBytesUsed += ramBytesUsed(filter);
|
||||
+ LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY * uniqueQueries.size();
|
||||
for (Query query : mostRecentlyUsedQueries) {
|
||||
recomputedRamBytesUsed += ramBytesUsed(query);
|
||||
}
|
||||
for (LeafCache leafCache : cache.values()) {
|
||||
recomputedRamBytesUsed += HASHTABLE_RAM_BYTES_PER_ENTRY * leafCache.cache.size();
|
||||
|
@ -351,41 +355,18 @@ public class LRUFilterCache implements FilterCache, Accountable {
|
|||
}
|
||||
|
||||
// pkg-private for testing
|
||||
// return the list of cached filters in LRU order
|
||||
synchronized List<Filter> cachedFilters() {
|
||||
return new ArrayList<>(mostRecentlyUsedFilters);
|
||||
// return the list of cached queries in LRU order
|
||||
synchronized List<Query> cachedQueries() {
|
||||
return new ArrayList<>(mostRecentlyUsedQueries);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Filter doCache(Filter filter, FilterCachingPolicy policy) {
|
||||
while (filter instanceof CachingWrapperFilter) {
|
||||
// should we throw an exception instead?
|
||||
filter = ((CachingWrapperFilter) filter).in;
|
||||
public Weight doCache(Weight weight, QueryCachingPolicy policy) {
|
||||
while (weight instanceof CachingWrapperWeight) {
|
||||
weight = ((CachingWrapperWeight) weight).in;
|
||||
}
|
||||
|
||||
return new CachingWrapperFilter(filter, policy);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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 calls
|
||||
* {@link #cacheImpl(DocIdSetIterator, org.apache.lucene.index.LeafReader)}
|
||||
* <p>Note: This method returns {@linkplain DocIdSet#EMPTY} if the given docIdSet
|
||||
* is <code>null</code> or if {@link DocIdSet#iterator()} return <code>null</code>. The empty
|
||||
* instance is use as a placeholder in the cache instead of the <code>null</code> value.
|
||||
*/
|
||||
protected DocIdSet docIdSetToCache(DocIdSet docIdSet, LeafReader reader) throws IOException {
|
||||
if (docIdSet == null || docIdSet.isCacheable()) {
|
||||
return docIdSet;
|
||||
} else {
|
||||
final DocIdSetIterator it = docIdSet.iterator();
|
||||
if (it == null) {
|
||||
return null;
|
||||
} else {
|
||||
return cacheImpl(it, reader);
|
||||
}
|
||||
}
|
||||
return new CachingWrapperWeight(weight, policy);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -401,15 +382,15 @@ public class LRUFilterCache implements FilterCache, Accountable {
|
|||
}
|
||||
|
||||
/**
|
||||
* Return the number of bytes used by the given filter. The default
|
||||
* implementation returns {@link Accountable#ramBytesUsed()} if the filter
|
||||
* Return the number of bytes used by the given query. The default
|
||||
* implementation returns {@link Accountable#ramBytesUsed()} if the query
|
||||
* implements {@link Accountable} and <code>1024</code> otherwise.
|
||||
*/
|
||||
protected long ramBytesUsed(Filter filter) {
|
||||
if (filter instanceof Accountable) {
|
||||
return ((Accountable) filter).ramBytesUsed();
|
||||
protected long ramBytesUsed(Query query) {
|
||||
if (query instanceof Accountable) {
|
||||
return ((Accountable) query).ramBytesUsed();
|
||||
}
|
||||
return FILTER_DEFAULT_RAM_BYTES_USED;
|
||||
return QUERY_DEFAULT_RAM_BYTES_USED;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -420,9 +401,9 @@ public class LRUFilterCache implements FilterCache, Accountable {
|
|||
}
|
||||
|
||||
/**
|
||||
* Return the total number of times that a {@link Filter} has been looked up
|
||||
* in this {@link FilterCache}. Note that this number is incremented once per
|
||||
* segment so running a cached filter only once will increment this counter
|
||||
* Return the total number of times that a {@link Query} has been looked up
|
||||
* in this {@link QueryCache}. Note that this number is incremented once per
|
||||
* segment so running a cached query only once will increment this counter
|
||||
* by the number of segments that are wrapped by the searcher.
|
||||
* Note that by definition, {@link #getTotalCount()} is the sum of
|
||||
* {@link #getHitCount()} and {@link #getMissCount()}.
|
||||
|
@ -434,7 +415,7 @@ public class LRUFilterCache implements FilterCache, Accountable {
|
|||
}
|
||||
|
||||
/**
|
||||
* Over the {@link #getTotalCount() total} number of times that a filter has
|
||||
* Over the {@link #getTotalCount() total} number of times that a query has
|
||||
* been looked up, return how many times a cached {@link DocIdSet} has been
|
||||
* found and returned.
|
||||
* @see #getTotalCount()
|
||||
|
@ -445,8 +426,8 @@ public class LRUFilterCache implements FilterCache, Accountable {
|
|||
}
|
||||
|
||||
/**
|
||||
* Over the {@link #getTotalCount() total} number of times that a filter has
|
||||
* been looked up, return how many times this filter was not contained in the
|
||||
* Over the {@link #getTotalCount() total} number of times that a query has
|
||||
* been looked up, return how many times this query was not contained in the
|
||||
* cache.
|
||||
* @see #getTotalCount()
|
||||
* @see #getHitCount()
|
||||
|
@ -469,8 +450,8 @@ public class LRUFilterCache implements FilterCache, Accountable {
|
|||
* Return the total number of cache entries that have been generated and put
|
||||
* in the cache. It is highly desirable to have a {@link #getHitCount() hit
|
||||
* count} that is much higher than the {@link #getCacheCount() cache count}
|
||||
* as the opposite would indicate that the filter cache makes efforts in order
|
||||
* to cache filters but then they do not get reused.
|
||||
* as the opposite would indicate that the query cache makes efforts in order
|
||||
* to cache queries but then they do not get reused.
|
||||
* @see #getCacheSize()
|
||||
* @see #getEvictionCount()
|
||||
*/
|
||||
|
@ -482,7 +463,7 @@ public class LRUFilterCache implements FilterCache, Accountable {
|
|||
* Return the number of cache entries that have been removed from the cache
|
||||
* either in order to stay under the maximum configured size/ram usage, or
|
||||
* because a segment has been closed. High numbers of evictions might mean
|
||||
* that filters are not reused or that the {@link FilterCachingPolicy
|
||||
* that queries are not reused or that the {@link QueryCachingPolicy
|
||||
* caching policy} caches too aggressively on NRT segments which get merged
|
||||
* early.
|
||||
* @see #getCacheCount()
|
||||
|
@ -496,7 +477,7 @@ public class LRUFilterCache implements FilterCache, Accountable {
|
|||
private class LeafCache implements Accountable {
|
||||
|
||||
private final Object key;
|
||||
private final Map<Filter, DocIdSet> cache;
|
||||
private final Map<Query, DocIdSet> cache;
|
||||
private volatile long ramBytesUsed;
|
||||
|
||||
LeafCache(Object key) {
|
||||
|
@ -507,27 +488,30 @@ public class LRUFilterCache implements FilterCache, Accountable {
|
|||
|
||||
private void onDocIdSetCache(long ramBytesUsed) {
|
||||
this.ramBytesUsed += ramBytesUsed;
|
||||
LRUFilterCache.this.onDocIdSetCache(key, ramBytesUsed);
|
||||
LRUQueryCache.this.onDocIdSetCache(key, ramBytesUsed);
|
||||
}
|
||||
|
||||
private void onDocIdSetEviction(long ramBytesUsed) {
|
||||
this.ramBytesUsed -= ramBytesUsed;
|
||||
LRUFilterCache.this.onDocIdSetEviction(key, 1, ramBytesUsed);
|
||||
LRUQueryCache.this.onDocIdSetEviction(key, 1, ramBytesUsed);
|
||||
}
|
||||
|
||||
DocIdSet get(Filter filter) {
|
||||
return cache.get(filter);
|
||||
DocIdSet get(Query query) {
|
||||
assert query == QueryCache.cacheKey(query);
|
||||
return cache.get(query);
|
||||
}
|
||||
|
||||
void putIfAbsent(Filter filter, DocIdSet set) {
|
||||
if (cache.putIfAbsent(filter, set) == null) {
|
||||
void putIfAbsent(Query query, DocIdSet set) {
|
||||
assert query == QueryCache.cacheKey(query);
|
||||
if (cache.putIfAbsent(query, set) == null) {
|
||||
// the set was actually put
|
||||
onDocIdSetCache(HASHTABLE_RAM_BYTES_PER_ENTRY + set.ramBytesUsed());
|
||||
}
|
||||
}
|
||||
|
||||
void remove(Filter filter) {
|
||||
DocIdSet removed = cache.remove(filter);
|
||||
void remove(Query query) {
|
||||
assert query == QueryCache.cacheKey(query);
|
||||
DocIdSet removed = cache.remove(query);
|
||||
if (removed != null) {
|
||||
onDocIdSetEviction(HASHTABLE_RAM_BYTES_PER_ENTRY + removed.ramBytesUsed());
|
||||
}
|
||||
|
@ -540,57 +524,106 @@ public class LRUFilterCache implements FilterCache, Accountable {
|
|||
|
||||
}
|
||||
|
||||
private class CachingWrapperFilter extends Filter {
|
||||
private class CachingWrapperWeight extends ConstantScoreWeight {
|
||||
|
||||
private final Filter in;
|
||||
private final FilterCachingPolicy policy;
|
||||
private final Weight in;
|
||||
private final QueryCachingPolicy policy;
|
||||
|
||||
CachingWrapperFilter(Filter in, FilterCachingPolicy policy) {
|
||||
CachingWrapperWeight(Weight in, QueryCachingPolicy policy) {
|
||||
super(in.getQuery());
|
||||
this.in = in;
|
||||
this.policy = policy;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
|
||||
Scorer scorer(LeafReaderContext context, Bits acceptDocs, float score) throws IOException {
|
||||
if (context.ord == 0) {
|
||||
policy.onUse(in);
|
||||
policy.onUse(getQuery());
|
||||
}
|
||||
|
||||
DocIdSet set = get(in, context);
|
||||
if (set == null) {
|
||||
// do not apply acceptDocs yet, we want the cached filter to not take them into account
|
||||
set = in.getDocIdSet(context, null);
|
||||
if (policy.shouldCache(in, context, set)) {
|
||||
set = docIdSetToCache(set, context.reader());
|
||||
if (set == null) {
|
||||
// null values are not supported
|
||||
set = DocIdSet.EMPTY;
|
||||
DocIdSet docIdSet = get(in.getQuery(), context);
|
||||
if (docIdSet == null) {
|
||||
if (policy.shouldCache(in.getQuery(), context)) {
|
||||
final Scorer scorer = in.scorer(context, null);
|
||||
if (scorer == null) {
|
||||
docIdSet = DocIdSet.EMPTY;
|
||||
} else {
|
||||
docIdSet = cacheImpl(scorer, context.reader());
|
||||
}
|
||||
// it might happen that another thread computed the same set in parallel
|
||||
// although this might incur some CPU overhead, it is probably better
|
||||
// this way than trying to lock and preventing other filters to be
|
||||
// computed at the same time?
|
||||
putIfAbsent(in, context, set);
|
||||
putIfAbsent(in.getQuery(), context, docIdSet);
|
||||
} else {
|
||||
return in.scorer(context, acceptDocs);
|
||||
}
|
||||
}
|
||||
return set == DocIdSet.EMPTY ? null : BitsFilteredDocIdSet.wrap(set, acceptDocs);
|
||||
|
||||
assert docIdSet != null;
|
||||
if (docIdSet == DocIdSet.EMPTY) {
|
||||
return null;
|
||||
}
|
||||
final DocIdSetIterator approximation = docIdSet.iterator();
|
||||
if (approximation == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
final DocIdSetIterator disi;
|
||||
final TwoPhaseIterator twoPhaseView;
|
||||
if (acceptDocs == null) {
|
||||
twoPhaseView = null;
|
||||
disi = approximation;
|
||||
} else {
|
||||
twoPhaseView = new TwoPhaseIterator() {
|
||||
|
||||
@Override
|
||||
public boolean matches() throws IOException {
|
||||
final int doc = approximation.docID();
|
||||
return acceptDocs.get(doc);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocIdSetIterator approximation() {
|
||||
return approximation;
|
||||
}
|
||||
};
|
||||
disi = TwoPhaseIterator.asDocIdSetIterator(twoPhaseView);
|
||||
}
|
||||
return new Scorer(this) {
|
||||
|
||||
@Override
|
||||
public TwoPhaseIterator asTwoPhaseIterator() {
|
||||
return twoPhaseView;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float score() throws IOException {
|
||||
return 0f;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int freq() throws IOException {
|
||||
return 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return disi.docID();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() throws IOException {
|
||||
return disi.nextDoc();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
return disi.advance(target);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long cost() {
|
||||
return disi.cost();
|
||||
}
|
||||
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
return obj instanceof CachingWrapperFilter
|
||||
&& in.equals(((CachingWrapperFilter) obj).in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return in.hashCode() ^ getClass().hashCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString(String field) {
|
||||
return "CachingWrapperFilter(" + in.toString(field) + ")";
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -18,17 +18,35 @@ package org.apache.lucene.search;
|
|||
*/
|
||||
|
||||
/**
|
||||
* A cache for filters.
|
||||
* A cache for queries.
|
||||
*
|
||||
* @see LRUFilterCache
|
||||
* @see LRUQueryCache
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public interface FilterCache {
|
||||
public interface QueryCache {
|
||||
|
||||
/**
|
||||
* Return a wrapper around the provided <code>filter</code> that will cache
|
||||
* {@link DocIdSet}s per-segment accordingly to the given <code>policy</code>.
|
||||
* Return a key for the given query that only takes matching documents into
|
||||
* account. Boosts will be ignored.
|
||||
* @lucene.internal
|
||||
*/
|
||||
Filter doCache(Filter filter, FilterCachingPolicy policy);
|
||||
public static Query cacheKey(Query query) {
|
||||
if (query.getBoost() == 1f) {
|
||||
return query;
|
||||
} else {
|
||||
Query key = query.clone();
|
||||
key.setBoost(1f);
|
||||
assert key == cacheKey(key);
|
||||
return key;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Return a wrapper around the provided <code>weight</code> that will cache
|
||||
* matching docs per-segment accordingly to the given <code>policy</code>.
|
||||
* NOTE: The returned weight will only be equivalent if scores are not needed.
|
||||
* @see Collector#needsScores()
|
||||
*/
|
||||
Weight doCache(Weight weight, QueryCachingPolicy policy);
|
||||
|
||||
}
|
|
@ -29,21 +29,21 @@ import org.apache.lucene.index.TieredMergePolicy;
|
|||
*
|
||||
* Implementations of this class must be thread-safe.
|
||||
*
|
||||
* @see UsageTrackingFilterCachingPolicy
|
||||
* @see LRUFilterCache
|
||||
* @see UsageTrackingQueryCachingPolicy
|
||||
* @see LRUQueryCache
|
||||
* @lucene.experimental
|
||||
*/
|
||||
// TODO: add APIs for integration with IndexWriter.IndexReaderWarmer
|
||||
public interface FilterCachingPolicy {
|
||||
public interface QueryCachingPolicy {
|
||||
|
||||
/** A simple policy that caches all the provided filters on all segments. */
|
||||
public static final FilterCachingPolicy ALWAYS_CACHE = new FilterCachingPolicy() {
|
||||
public static final QueryCachingPolicy ALWAYS_CACHE = new QueryCachingPolicy() {
|
||||
|
||||
@Override
|
||||
public void onUse(Filter filter) {}
|
||||
public void onUse(Query query) {}
|
||||
|
||||
@Override
|
||||
public boolean shouldCache(Filter filter, LeafReaderContext context, DocIdSet set) throws IOException {
|
||||
public boolean shouldCache(Query query, LeafReaderContext context) throws IOException {
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -54,13 +54,13 @@ public interface FilterCachingPolicy {
|
|||
* execution time of queries and are also more likely to stay around longer
|
||||
* than small segments, which makes them more interesting for caching.
|
||||
*/
|
||||
public static class CacheOnLargeSegments implements FilterCachingPolicy {
|
||||
public static class CacheOnLargeSegments implements QueryCachingPolicy {
|
||||
|
||||
/** {@link CacheOnLargeSegments} instance that only caches on segments that
|
||||
* account for more than 3% of the total index size. This should guarantee
|
||||
* that all segments from the upper {@link TieredMergePolicy tier} will be
|
||||
* cached while ensuring that at most <tt>33</tt> segments can make it to
|
||||
* the cache (given that some implementations such as {@link LRUFilterCache}
|
||||
* the cache (given that some implementations such as {@link LRUQueryCache}
|
||||
* perform better when the number of cached segments is low). */
|
||||
public static final CacheOnLargeSegments DEFAULT = new CacheOnLargeSegments(0.03f);
|
||||
|
||||
|
@ -80,10 +80,10 @@ public interface FilterCachingPolicy {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void onUse(Filter filter) {}
|
||||
public void onUse(Query query) {}
|
||||
|
||||
@Override
|
||||
public boolean shouldCache(Filter filter, LeafReaderContext context, DocIdSet set) throws IOException {
|
||||
public boolean shouldCache(Query query, LeafReaderContext context) throws IOException {
|
||||
final IndexReaderContext topLevelContext = ReaderUtil.getTopLevelContext(context);
|
||||
final float sizeRatio = (float) context.reader().maxDoc() / topLevelContext.reader().maxDoc();
|
||||
return sizeRatio >= minSizeRatio;
|
||||
|
@ -94,7 +94,7 @@ public interface FilterCachingPolicy {
|
|||
/** Callback that is called every time that a cached filter is used.
|
||||
* This is typically useful if the policy wants to track usage statistics
|
||||
* in order to make decisions. */
|
||||
void onUse(Filter filter);
|
||||
void onUse(Query query);
|
||||
|
||||
/** Whether the given {@link DocIdSet} should be cached on a given segment.
|
||||
* This method will be called on each leaf context to know if the filter
|
||||
|
@ -102,6 +102,6 @@ public interface FilterCachingPolicy {
|
|||
* attempt to load a {@link DocIdSet} from the cache. If it is not cached
|
||||
* yet and this method returns <tt>true</tt> then a cache entry will be
|
||||
* generated. Otherwise an uncached set will be returned. */
|
||||
boolean shouldCache(Filter filter, LeafReaderContext context, DocIdSet set) throws IOException;
|
||||
boolean shouldCache(Query query, LeafReaderContext context) throws IOException;
|
||||
|
||||
}
|
|
@ -1,136 +0,0 @@
|
|||
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 java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.FrequencyTrackingRingBuffer;
|
||||
|
||||
|
||||
/**
|
||||
* A {@link FilterCachingPolicy} that tracks usage statistics of recently-used
|
||||
* filters in order to decide on which filters are worth caching.
|
||||
*
|
||||
* It also uses some heuristics on segments, filters and the doc id sets that
|
||||
* they produce in order to cache more aggressively when the execution cost
|
||||
* significantly outweighs the caching overhead.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public final class UsageTrackingFilterCachingPolicy implements FilterCachingPolicy {
|
||||
|
||||
// the hash code that we use as a sentinel in the ring buffer.
|
||||
private static final int SENTINEL = Integer.MIN_VALUE;
|
||||
|
||||
static boolean isCostly(Filter filter) {
|
||||
// This does not measure the cost of iterating over the filter (for this we
|
||||
// already have the DocIdSetIterator#cost API) but the cost to build the
|
||||
// DocIdSet in the first place
|
||||
return filter instanceof QueryWrapperFilter && ((QueryWrapperFilter) filter).getQuery() instanceof MultiTermQuery;
|
||||
}
|
||||
|
||||
static boolean isCheapToCache(DocIdSet set) {
|
||||
// the produced doc set is already cacheable, so caching has no
|
||||
// overhead at all. TODO: extend this to sets whose iterators have a low
|
||||
// cost?
|
||||
return set == null || set.isCacheable();
|
||||
}
|
||||
|
||||
private final FilterCachingPolicy.CacheOnLargeSegments segmentPolicy;
|
||||
private final FrequencyTrackingRingBuffer recentlyUsedFilters;
|
||||
private final int minFrequencyCostlyFilters;
|
||||
private final int minFrequencyCheapFilters;
|
||||
private final int minFrequencyOtherFilters;
|
||||
|
||||
/**
|
||||
* Create a new instance.
|
||||
*
|
||||
* @param minSizeRatio the minimum size ratio for segments to be cached, see {@link FilterCachingPolicy.CacheOnLargeSegments}
|
||||
* @param historySize the number of recently used filters to track
|
||||
* @param minFrequencyCostlyFilters how many times filters whose {@link Filter#getDocIdSet(LeafReaderContext, Bits) getDocIdSet} method is expensive should have been seen before being cached
|
||||
* @param minFrequencyCheapFilters how many times filters that produce {@link DocIdSet}s that are cheap to cached should have been seen before being cached
|
||||
* @param minFrequencyOtherFilters how many times other filters should have been seen before being cached
|
||||
*/
|
||||
public UsageTrackingFilterCachingPolicy(
|
||||
float minSizeRatio,
|
||||
int historySize,
|
||||
int minFrequencyCostlyFilters,
|
||||
int minFrequencyCheapFilters,
|
||||
int minFrequencyOtherFilters) {
|
||||
this(new FilterCachingPolicy.CacheOnLargeSegments(minSizeRatio), historySize,
|
||||
minFrequencyCostlyFilters, minFrequencyCheapFilters, minFrequencyOtherFilters);
|
||||
}
|
||||
|
||||
/** Create a new instance with sensible defaults. */
|
||||
public UsageTrackingFilterCachingPolicy() {
|
||||
// we track the most 256 recently-used filters and cache filters that are
|
||||
// expensive to build or cheap to cache after we have seen them twice, and
|
||||
// cache regular filters after we have seen them 5 times
|
||||
this(FilterCachingPolicy.CacheOnLargeSegments.DEFAULT, 256, 2, 2, 5);
|
||||
}
|
||||
|
||||
private UsageTrackingFilterCachingPolicy(
|
||||
FilterCachingPolicy.CacheOnLargeSegments segmentPolicy,
|
||||
int historySize,
|
||||
int minFrequencyCostlyFilters,
|
||||
int minFrequencyCheapFilters,
|
||||
int minFrequencyOtherFilters) {
|
||||
this.segmentPolicy = segmentPolicy;
|
||||
if (minFrequencyOtherFilters < minFrequencyCheapFilters || minFrequencyOtherFilters < minFrequencyCheapFilters) {
|
||||
throw new IllegalArgumentException("it does not make sense to cache regular filters more aggressively than filters that are costly to produce or cheap to cache");
|
||||
}
|
||||
if (minFrequencyCheapFilters > historySize || minFrequencyCostlyFilters > historySize || minFrequencyOtherFilters > historySize) {
|
||||
throw new IllegalArgumentException("The minimum frequencies should be less than the size of the history of filters that are being tracked");
|
||||
}
|
||||
this.recentlyUsedFilters = new FrequencyTrackingRingBuffer(historySize, SENTINEL);
|
||||
this.minFrequencyCostlyFilters = minFrequencyCostlyFilters;
|
||||
this.minFrequencyCheapFilters = minFrequencyCheapFilters;
|
||||
this.minFrequencyOtherFilters = minFrequencyOtherFilters;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onUse(Filter filter) {
|
||||
// we only track hash codes, which
|
||||
synchronized (this) {
|
||||
recentlyUsedFilters.add(filter.hashCode());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean shouldCache(Filter filter, LeafReaderContext context, DocIdSet set) throws IOException {
|
||||
if (segmentPolicy.shouldCache(filter, context, set) == false) {
|
||||
return false;
|
||||
}
|
||||
final int frequency;
|
||||
synchronized (this) {
|
||||
frequency = recentlyUsedFilters.frequency(filter.hashCode());
|
||||
}
|
||||
if (frequency >= minFrequencyOtherFilters) {
|
||||
return true;
|
||||
} else if (isCostly(filter) && frequency >= minFrequencyCostlyFilters) {
|
||||
return true;
|
||||
} else if (isCheapToCache(set) && frequency >= minFrequencyCheapFilters) {
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,119 @@
|
|||
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 java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.util.FrequencyTrackingRingBuffer;
|
||||
|
||||
|
||||
/**
|
||||
* A {@link QueryCachingPolicy} that tracks usage statistics of recently-used
|
||||
* filters in order to decide on which filters are worth caching.
|
||||
*
|
||||
* It also uses some heuristics on segments, filters and the doc id sets that
|
||||
* they produce in order to cache more aggressively when the execution cost
|
||||
* significantly outweighs the caching overhead.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public final class UsageTrackingQueryCachingPolicy implements QueryCachingPolicy {
|
||||
|
||||
// the hash code that we use as a sentinel in the ring buffer.
|
||||
private static final int SENTINEL = Integer.MIN_VALUE;
|
||||
|
||||
static boolean isCostly(Query query) {
|
||||
// This does not measure the cost of iterating over the filter (for this we
|
||||
// already have the DocIdSetIterator#cost API) but the cost to build the
|
||||
// DocIdSet in the first place
|
||||
return query instanceof MultiTermQuery ||
|
||||
query instanceof MultiTermQueryConstantScoreWrapper;
|
||||
}
|
||||
|
||||
static boolean isCheap(Query query) {
|
||||
// same for cheap queries
|
||||
// these queries are so cheap that they usually do not need caching
|
||||
return query instanceof TermQuery;
|
||||
}
|
||||
|
||||
private final QueryCachingPolicy.CacheOnLargeSegments segmentPolicy;
|
||||
private final FrequencyTrackingRingBuffer recentlyUsedFilters;
|
||||
|
||||
/**
|
||||
* Create a new instance.
|
||||
*
|
||||
* @param minSizeRatio the minimum size ratio for segments to be cached, see {@link QueryCachingPolicy.CacheOnLargeSegments}
|
||||
* @param historySize the number of recently used filters to track
|
||||
*/
|
||||
public UsageTrackingQueryCachingPolicy(
|
||||
float minSizeRatio,
|
||||
int historySize) {
|
||||
this(new QueryCachingPolicy.CacheOnLargeSegments(minSizeRatio), historySize);
|
||||
}
|
||||
|
||||
/** Create a new instance with an history size of 256. */
|
||||
public UsageTrackingQueryCachingPolicy() {
|
||||
this(QueryCachingPolicy.CacheOnLargeSegments.DEFAULT, 256);
|
||||
}
|
||||
|
||||
private UsageTrackingQueryCachingPolicy(
|
||||
QueryCachingPolicy.CacheOnLargeSegments segmentPolicy,
|
||||
int historySize) {
|
||||
this.segmentPolicy = segmentPolicy;
|
||||
this.recentlyUsedFilters = new FrequencyTrackingRingBuffer(historySize, SENTINEL);
|
||||
}
|
||||
|
||||
/**
|
||||
* For a given query, return how many times it should appear in the history
|
||||
* before being cached.
|
||||
*/
|
||||
protected int minFrequencyToCache(Query query) {
|
||||
if (isCostly(query)) {
|
||||
return 2;
|
||||
} else if (isCheap(query)) {
|
||||
return 20;
|
||||
} else {
|
||||
// default: cache after the filter has been seen 5 times
|
||||
return 5;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onUse(Query query) {
|
||||
// we only track hash codes, which
|
||||
synchronized (this) {
|
||||
recentlyUsedFilters.add(QueryCache.cacheKey(query).hashCode());
|
||||
}
|
||||
}
|
||||
|
||||
synchronized int frequency(Query query) {
|
||||
return recentlyUsedFilters.frequency(QueryCache.cacheKey(query).hashCode());
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean shouldCache(Query query, LeafReaderContext context) throws IOException {
|
||||
if (segmentPolicy.shouldCache(query, context) == false) {
|
||||
return false;
|
||||
}
|
||||
final int frequency = frequency(query);
|
||||
final int minFrequency = minFrequencyToCache(query);
|
||||
return frequency >= minFrequency;
|
||||
}
|
||||
|
||||
}
|
|
@ -61,12 +61,6 @@ public class BitDocIdSet extends DocIdSet {
|
|||
return set;
|
||||
}
|
||||
|
||||
/** This DocIdSet implementation is cacheable. */
|
||||
@Override
|
||||
public boolean isCacheable() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return BASE_RAM_BYTES_USED + set.ramBytesUsed();
|
||||
|
|
|
@ -41,11 +41,6 @@ public final class NotDocIdSet extends DocIdSet {
|
|||
this.in = in;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isCacheable() {
|
||||
return in.isCacheable();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits bits() throws IOException {
|
||||
final Bits inBits = in.bits();
|
||||
|
|
|
@ -242,11 +242,6 @@ public class RoaringDocIdSet extends DocIdSet {
|
|||
this.cardinality = cardinality;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isCacheable() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return ramBytesUsed;
|
||||
|
|
|
@ -111,14 +111,6 @@ public class EliasFanoDocIdSet extends DocIdSet {
|
|||
};
|
||||
}
|
||||
|
||||
/** This DocIdSet implementation is cacheable.
|
||||
* @return <code>true</code>
|
||||
*/
|
||||
@Override
|
||||
public boolean isCacheable() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object other) {
|
||||
return ((other instanceof EliasFanoDocIdSet))
|
||||
|
|
|
@ -327,6 +327,10 @@ public class TestOmitTf extends LuceneTestCase {
|
|||
new CountingHitCollector() {
|
||||
private Scorer scorer;
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true;
|
||||
}
|
||||
@Override
|
||||
public final void setScorer(Scorer scorer) {
|
||||
this.scorer = scorer;
|
||||
}
|
||||
|
@ -345,6 +349,10 @@ public class TestOmitTf extends LuceneTestCase {
|
|||
new CountingHitCollector() {
|
||||
private Scorer scorer;
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true;
|
||||
}
|
||||
@Override
|
||||
public final void setScorer(Scorer scorer) {
|
||||
this.scorer = scorer;
|
||||
}
|
||||
|
@ -366,6 +374,10 @@ public class TestOmitTf extends LuceneTestCase {
|
|||
new CountingHitCollector() {
|
||||
private Scorer scorer;
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true;
|
||||
}
|
||||
@Override
|
||||
public final void setScorer(Scorer scorer) {
|
||||
this.scorer = scorer;
|
||||
}
|
||||
|
@ -385,6 +397,10 @@ public class TestOmitTf extends LuceneTestCase {
|
|||
new CountingHitCollector() {
|
||||
private Scorer scorer;
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true;
|
||||
}
|
||||
@Override
|
||||
public final void setScorer(Scorer scorer) {
|
||||
this.scorer = scorer;
|
||||
}
|
||||
|
|
|
@ -1,449 +0,0 @@
|
|||
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 java.io.IOException;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.StringField;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.SerialMergeScheduler;
|
||||
import org.apache.lucene.index.SlowCompositeReaderWrapper;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BitDocIdSet;
|
||||
import org.apache.lucene.util.FixedBitSet;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.RoaringDocIdSet;
|
||||
|
||||
public class TestCachingWrapperFilter extends LuceneTestCase {
|
||||
Directory dir;
|
||||
DirectoryReader ir;
|
||||
IndexSearcher is;
|
||||
RandomIndexWriter iw;
|
||||
|
||||
@Override
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
dir = newDirectory();
|
||||
iw = new RandomIndexWriter(random(), dir);
|
||||
Document doc = new Document();
|
||||
Field idField = new StringField("id", "", Field.Store.NO);
|
||||
doc.add(idField);
|
||||
// add 500 docs with id 0..499
|
||||
for (int i = 0; i < 500; i++) {
|
||||
idField.setStringValue(Integer.toString(i));
|
||||
iw.addDocument(doc);
|
||||
}
|
||||
// delete 20 of them
|
||||
for (int i = 0; i < 20; i++) {
|
||||
iw.deleteDocuments(new Term("id", Integer.toString(random().nextInt(iw.maxDoc()))));
|
||||
}
|
||||
ir = iw.getReader();
|
||||
is = newSearcher(ir);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void tearDown() throws Exception {
|
||||
iw.close();
|
||||
IOUtils.close(ir, dir);
|
||||
super.tearDown();
|
||||
}
|
||||
|
||||
private void assertFilterEquals(Filter f1, Filter f2) throws Exception {
|
||||
Query query = new MatchAllDocsQuery();
|
||||
TopDocs hits1 = is.search(new FilteredQuery(query, f1), ir.maxDoc());
|
||||
TopDocs hits2 = is.search(new FilteredQuery(query, f2), ir.maxDoc());
|
||||
assertEquals(hits1.totalHits, hits2.totalHits);
|
||||
CheckHits.checkEqual(query, hits1.scoreDocs, hits2.scoreDocs);
|
||||
// now do it again to confirm caching works
|
||||
TopDocs hits3 = is.search(new FilteredQuery(query, f1), ir.maxDoc());
|
||||
TopDocs hits4 = is.search(new FilteredQuery(query, f2), ir.maxDoc());
|
||||
assertEquals(hits3.totalHits, hits4.totalHits);
|
||||
CheckHits.checkEqual(query, hits3.scoreDocs, hits4.scoreDocs);
|
||||
}
|
||||
|
||||
/** test null iterator */
|
||||
public void testEmpty() throws Exception {
|
||||
Query query = new BooleanQuery();
|
||||
Filter expected = new QueryWrapperFilter(query);
|
||||
Filter actual = new CachingWrapperFilter(expected, MAYBE_CACHE_POLICY);
|
||||
assertFilterEquals(expected, actual);
|
||||
}
|
||||
|
||||
/** test iterator returns NO_MORE_DOCS */
|
||||
public void testEmpty2() throws Exception {
|
||||
BooleanQuery query = new BooleanQuery();
|
||||
query.add(new TermQuery(new Term("id", "0")), BooleanClause.Occur.MUST);
|
||||
query.add(new TermQuery(new Term("id", "0")), BooleanClause.Occur.MUST_NOT);
|
||||
Filter expected = new QueryWrapperFilter(query);
|
||||
Filter actual = new CachingWrapperFilter(expected, MAYBE_CACHE_POLICY);
|
||||
assertFilterEquals(expected, actual);
|
||||
}
|
||||
|
||||
/** test null docidset */
|
||||
public void testEmpty3() throws Exception {
|
||||
Filter expected = new QueryWrapperFilter(new PrefixQuery(new Term("bogusField", "bogusVal")));
|
||||
Filter actual = new CachingWrapperFilter(expected, MAYBE_CACHE_POLICY);
|
||||
assertFilterEquals(expected, actual);
|
||||
}
|
||||
|
||||
/** test iterator returns single document */
|
||||
public void testSingle() throws Exception {
|
||||
for (int i = 0; i < 10; i++) {
|
||||
int id = random().nextInt(ir.maxDoc());
|
||||
Query query = new TermQuery(new Term("id", Integer.toString(id)));
|
||||
Filter expected = new QueryWrapperFilter(query);
|
||||
Filter actual = new CachingWrapperFilter(expected, MAYBE_CACHE_POLICY);
|
||||
assertFilterEquals(expected, actual);
|
||||
}
|
||||
}
|
||||
|
||||
/** test sparse filters (match single documents) */
|
||||
public void testSparse() throws Exception {
|
||||
for (int i = 0; i < 10; i++) {
|
||||
int id_start = random().nextInt(ir.maxDoc()-1);
|
||||
int id_end = id_start + 1;
|
||||
Query query = TermRangeQuery.newStringRange("id",
|
||||
Integer.toString(id_start), Integer.toString(id_end), true, true);
|
||||
Filter expected = new QueryWrapperFilter(query);
|
||||
Filter actual = new CachingWrapperFilter(expected, MAYBE_CACHE_POLICY);
|
||||
assertFilterEquals(expected, actual);
|
||||
}
|
||||
}
|
||||
|
||||
/** test dense filters (match entire index) */
|
||||
public void testDense() throws Exception {
|
||||
Query query = new MatchAllDocsQuery();
|
||||
Filter expected = new QueryWrapperFilter(query);
|
||||
Filter actual = new CachingWrapperFilter(expected, FilterCachingPolicy.ALWAYS_CACHE);
|
||||
assertFilterEquals(expected, actual);
|
||||
}
|
||||
|
||||
public void testCachingWorks() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
|
||||
writer.close();
|
||||
|
||||
IndexReader reader = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(dir));
|
||||
LeafReaderContext context = (LeafReaderContext) reader.getContext();
|
||||
MockFilter filter = new MockFilter();
|
||||
CachingWrapperFilter cacher = new CachingWrapperFilter(filter, FilterCachingPolicy.ALWAYS_CACHE);
|
||||
|
||||
// first time, nested filter is called
|
||||
DocIdSet strongRef = 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, context.reader().getLiveDocs());
|
||||
|
||||
// second time, nested filter should not be called
|
||||
filter.clear();
|
||||
cacher.getDocIdSet(context, context.reader().getLiveDocs());
|
||||
assertFalse("second time", filter.wasCalled());
|
||||
|
||||
reader.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testNullDocIdSet() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
|
||||
writer.close();
|
||||
|
||||
IndexReader reader = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(dir));
|
||||
LeafReaderContext context = (LeafReaderContext) reader.getContext();
|
||||
|
||||
final Filter filter = new Filter() {
|
||||
@Override
|
||||
public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) {
|
||||
return null;
|
||||
}
|
||||
@Override
|
||||
public String toString(String field) {
|
||||
return "nullDocIdSetFilter";
|
||||
}
|
||||
};
|
||||
CachingWrapperFilter cacher = new CachingWrapperFilter(filter, MAYBE_CACHE_POLICY);
|
||||
|
||||
// the caching filter should return the empty set constant
|
||||
assertNull(cacher.getDocIdSet(context, context.reader().getLiveDocs()));
|
||||
|
||||
reader.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testNullDocIdSetIterator() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
|
||||
writer.close();
|
||||
|
||||
IndexReader reader = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(dir));
|
||||
LeafReaderContext context = (LeafReaderContext) reader.getContext();
|
||||
|
||||
final Filter filter = new Filter() {
|
||||
@Override
|
||||
public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) {
|
||||
return new DocIdSet() {
|
||||
@Override
|
||||
public DocIdSetIterator iterator() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return 0L;
|
||||
}
|
||||
};
|
||||
}
|
||||
@Override
|
||||
public String toString(String field) {
|
||||
return "nullDocIdSetIteratorFilter";
|
||||
}
|
||||
};
|
||||
CachingWrapperFilter cacher = new CachingWrapperFilter(filter, FilterCachingPolicy.ALWAYS_CACHE);
|
||||
|
||||
// the caching filter should return the empty set constant
|
||||
assertNull(cacher.getDocIdSet(context, context.reader().getLiveDocs()));
|
||||
|
||||
reader.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
private static void assertDocIdSetCacheable(IndexReader reader, Filter filter, boolean shouldCacheable) throws IOException {
|
||||
assertTrue(reader.getContext() instanceof LeafReaderContext);
|
||||
LeafReaderContext context = (LeafReaderContext) reader.getContext();
|
||||
final CachingWrapperFilter cacher = new CachingWrapperFilter(filter, FilterCachingPolicy.ALWAYS_CACHE);
|
||||
final DocIdSet originalSet = filter.getDocIdSet(context, context.reader().getLiveDocs());
|
||||
final DocIdSet cachedSet = cacher.getDocIdSet(context, context.reader().getLiveDocs());
|
||||
if (originalSet == null) {
|
||||
assertNull(cachedSet);
|
||||
}
|
||||
if (cachedSet == null) {
|
||||
assertTrue(originalSet == null || originalSet.iterator() == null);
|
||||
} else {
|
||||
assertTrue(cachedSet.isCacheable());
|
||||
assertEquals(shouldCacheable, originalSet.isCacheable());
|
||||
//System.out.println("Original: "+originalSet.getClass().getName()+" -- cached: "+cachedSet.getClass().getName());
|
||||
if (originalSet.isCacheable()) {
|
||||
assertEquals("Cached DocIdSet must be of same class like uncached, if cacheable", originalSet.getClass(), cachedSet.getClass());
|
||||
} else {
|
||||
assertTrue("Cached DocIdSet must be a RoaringDocIdSet if the original one was not cacheable", cachedSet instanceof RoaringDocIdSet || cachedSet == null);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void testIsCacheAble() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
|
||||
writer.addDocument(new Document());
|
||||
writer.close();
|
||||
|
||||
IndexReader reader = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(dir));
|
||||
|
||||
// not cacheable:
|
||||
assertDocIdSetCacheable(reader, new QueryWrapperFilter(new TermQuery(new Term("test","value"))), false);
|
||||
// returns default empty docidset, always cacheable:
|
||||
assertDocIdSetCacheable(reader, new Filter() {
|
||||
@Override
|
||||
public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) {
|
||||
return null;
|
||||
}
|
||||
@Override
|
||||
public String toString(String field) {
|
||||
return "cacheableFilter";
|
||||
}
|
||||
}, true);
|
||||
// is cacheable:
|
||||
assertDocIdSetCacheable(reader, new QueryWrapperFilter(NumericRangeQuery.newIntRange("test", 10, 20, true, true)), false);
|
||||
// a fixedbitset filter is always cacheable
|
||||
assertDocIdSetCacheable(reader, new Filter() {
|
||||
@Override
|
||||
public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) {
|
||||
return new BitDocIdSet(new FixedBitSet(context.reader().maxDoc()));
|
||||
}
|
||||
@Override
|
||||
public String toString(String field) {
|
||||
return "cacheableFilter";
|
||||
}
|
||||
}, true);
|
||||
|
||||
reader.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testEnforceDeletions() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter writer = new RandomIndexWriter(
|
||||
random(),
|
||||
dir,
|
||||
newIndexWriterConfig(new MockAnalyzer(random())).
|
||||
setMergeScheduler(new SerialMergeScheduler()).
|
||||
// asserts below requires no unexpected merges:
|
||||
setMergePolicy(newLogMergePolicy(10))
|
||||
);
|
||||
|
||||
// NOTE: cannot use writer.getReader because RIW (on
|
||||
// flipping a coin) may give us a newly opened reader,
|
||||
// but we use .reopen on this reader below and expect to
|
||||
// (must) get an NRT reader:
|
||||
DirectoryReader reader = DirectoryReader.open(writer.w, true);
|
||||
// same reason we don't wrap?
|
||||
IndexSearcher searcher = newSearcher(reader, false);
|
||||
|
||||
// add a doc, refresh the reader, and check that it's there
|
||||
Document doc = new Document();
|
||||
doc.add(newStringField("id", "1", Field.Store.YES));
|
||||
writer.addDocument(doc);
|
||||
|
||||
reader = refreshReader(reader);
|
||||
searcher = newSearcher(reader, false);
|
||||
|
||||
TopDocs docs = searcher.search(new MatchAllDocsQuery(), 1);
|
||||
assertEquals("Should find a hit...", 1, docs.totalHits);
|
||||
|
||||
final Filter startFilter = new QueryWrapperFilter(new TermQuery(new Term("id", "1")));
|
||||
|
||||
CachingWrapperFilter filter = new CachingWrapperFilter(startFilter, FilterCachingPolicy.ALWAYS_CACHE);
|
||||
|
||||
docs = searcher.search(new FilteredQuery(new MatchAllDocsQuery(), filter), 1);
|
||||
assertTrue(filter.ramBytesUsed() > 0);
|
||||
|
||||
assertEquals("[query + filter] Should find a hit...", 1, docs.totalHits);
|
||||
|
||||
Query constantScore = new ConstantScoreQuery(filter);
|
||||
docs = searcher.search(constantScore, 1);
|
||||
assertEquals("[just filter] Should find a hit...", 1, docs.totalHits);
|
||||
|
||||
// make sure we get a cache hit when we reopen reader
|
||||
// that had no change to deletions
|
||||
|
||||
// fake delete (deletes nothing):
|
||||
writer.deleteDocuments(new Term("foo", "bar"));
|
||||
|
||||
IndexReader oldReader = reader;
|
||||
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);
|
||||
|
||||
// cache hit:
|
||||
assertEquals(missCount, filter.missCount);
|
||||
|
||||
// now delete the doc, refresh the reader, and see that it's not there
|
||||
writer.deleteDocuments(new Term("id", "1"));
|
||||
|
||||
// NOTE: important to hold ref here so GC doesn't clear
|
||||
// the cache entry! Else the assert below may sometimes
|
||||
// fail:
|
||||
oldReader = reader;
|
||||
reader = refreshReader(reader);
|
||||
|
||||
searcher = newSearcher(reader, false);
|
||||
|
||||
missCount = filter.missCount;
|
||||
docs = searcher.search(new FilteredQuery(new MatchAllDocsQuery(), filter), 1);
|
||||
assertEquals("[query + filter] Should *not* find a hit...", 0, docs.totalHits);
|
||||
|
||||
// cache hit
|
||||
assertEquals(missCount, filter.missCount);
|
||||
docs = searcher.search(constantScore, 1);
|
||||
assertEquals("[just filter] Should *not* find a hit...", 0, docs.totalHits);
|
||||
|
||||
// apply deletes dynamically:
|
||||
filter = new CachingWrapperFilter(startFilter, FilterCachingPolicy.ALWAYS_CACHE);
|
||||
writer.addDocument(doc);
|
||||
reader = refreshReader(reader);
|
||||
searcher = newSearcher(reader, false);
|
||||
|
||||
docs = searcher.search(new FilteredQuery(new MatchAllDocsQuery(), filter), 1);
|
||||
assertEquals("[query + filter] Should find a hit...", 1, docs.totalHits);
|
||||
missCount = filter.missCount;
|
||||
assertTrue(missCount > 0);
|
||||
constantScore = new ConstantScoreQuery(filter);
|
||||
docs = searcher.search(constantScore, 1);
|
||||
assertEquals("[just filter] Should find a hit...", 1, docs.totalHits);
|
||||
assertEquals(missCount, filter.missCount);
|
||||
|
||||
writer.addDocument(doc);
|
||||
|
||||
// NOTE: important to hold ref here so GC doesn't clear
|
||||
// the cache entry! Else the assert below may sometimes
|
||||
// fail:
|
||||
oldReader = reader;
|
||||
|
||||
reader = refreshReader(reader);
|
||||
searcher = newSearcher(reader, false);
|
||||
|
||||
docs = searcher.search(new FilteredQuery(new MatchAllDocsQuery(), filter), 1);
|
||||
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...", 2, 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 = newSearcher(reader, false);
|
||||
|
||||
docs = searcher.search(new FilteredQuery(new MatchAllDocsQuery(), filter), 1);
|
||||
assertEquals("[query + filter] Should *not* find a hit...", 0, docs.totalHits);
|
||||
// CWF reused the same entry (it dynamically applied the deletes):
|
||||
assertEquals(missCount, filter.missCount);
|
||||
|
||||
docs = searcher.search(constantScore, 1);
|
||||
assertEquals("[just filter] Should *not* find a hit...", 0, docs.totalHits);
|
||||
// CWF reused the same entry (it dynamically applied the deletes):
|
||||
assertEquals(missCount, filter.missCount);
|
||||
|
||||
// NOTE: silliness to make sure JRE does not eliminate
|
||||
// our holding onto oldReader to prevent
|
||||
// CachingWrapperFilter's WeakHashMap from dropping the
|
||||
// entry:
|
||||
assertTrue(oldReader != null);
|
||||
|
||||
reader.close();
|
||||
writer.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
private static DirectoryReader refreshReader(DirectoryReader reader) throws IOException {
|
||||
DirectoryReader oldReader = reader;
|
||||
reader = DirectoryReader.openIfChanged(reader);
|
||||
if (reader != null) {
|
||||
oldReader.close();
|
||||
return reader;
|
||||
} else {
|
||||
return oldReader;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,309 @@
|
|||
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 java.io.IOException;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.StringField;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.SerialMergeScheduler;
|
||||
import org.apache.lucene.index.SlowCompositeReaderWrapper;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
public class TestCachingWrapperQuery extends LuceneTestCase {
|
||||
Directory dir;
|
||||
DirectoryReader ir;
|
||||
IndexSearcher is;
|
||||
RandomIndexWriter iw;
|
||||
|
||||
@Override
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
dir = newDirectory();
|
||||
iw = new RandomIndexWriter(random(), dir);
|
||||
Document doc = new Document();
|
||||
Field idField = new StringField("id", "", Field.Store.NO);
|
||||
doc.add(idField);
|
||||
// add 500 docs with id 0..499
|
||||
for (int i = 0; i < 500; i++) {
|
||||
idField.setStringValue(Integer.toString(i));
|
||||
iw.addDocument(doc);
|
||||
}
|
||||
// delete 20 of them
|
||||
for (int i = 0; i < 20; i++) {
|
||||
iw.deleteDocuments(new Term("id", Integer.toString(random().nextInt(iw.maxDoc()))));
|
||||
}
|
||||
ir = iw.getReader();
|
||||
is = newSearcher(ir);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void tearDown() throws Exception {
|
||||
iw.close();
|
||||
IOUtils.close(ir, dir);
|
||||
super.tearDown();
|
||||
}
|
||||
|
||||
private void assertQueryEquals(Query f1, Query f2) throws Exception {
|
||||
// wrap into CSQ so that scores are not needed
|
||||
TopDocs hits1 = is.search(new ConstantScoreQuery(f1), ir.maxDoc());
|
||||
TopDocs hits2 = is.search(new ConstantScoreQuery(f2), ir.maxDoc());
|
||||
assertEquals(hits1.totalHits, hits2.totalHits);
|
||||
CheckHits.checkEqual(f1, hits1.scoreDocs, hits2.scoreDocs);
|
||||
// now do it again to confirm caching works
|
||||
TopDocs hits3 = is.search(new ConstantScoreQuery(f1), ir.maxDoc());
|
||||
TopDocs hits4 = is.search(new ConstantScoreQuery(f2), ir.maxDoc());
|
||||
assertEquals(hits3.totalHits, hits4.totalHits);
|
||||
CheckHits.checkEqual(f1, hits3.scoreDocs, hits4.scoreDocs);
|
||||
}
|
||||
|
||||
/** test null iterator */
|
||||
public void testEmpty() throws Exception {
|
||||
Query expected = new BooleanQuery();
|
||||
Query cached = new CachingWrapperQuery(expected, MAYBE_CACHE_POLICY);
|
||||
assertQueryEquals(expected, cached);
|
||||
}
|
||||
|
||||
/** test iterator returns NO_MORE_DOCS */
|
||||
public void testEmpty2() throws Exception {
|
||||
BooleanQuery expected = new BooleanQuery();
|
||||
expected.add(new TermQuery(new Term("id", "0")), BooleanClause.Occur.MUST);
|
||||
expected.add(new TermQuery(new Term("id", "0")), BooleanClause.Occur.MUST_NOT);
|
||||
Query cached = new CachingWrapperQuery(expected, MAYBE_CACHE_POLICY);
|
||||
assertQueryEquals(expected, cached);
|
||||
}
|
||||
|
||||
/** test iterator returns single document */
|
||||
public void testSingle() throws Exception {
|
||||
for (int i = 0; i < 10; i++) {
|
||||
int id = random().nextInt(ir.maxDoc());
|
||||
Query expected = new TermQuery(new Term("id", Integer.toString(id)));
|
||||
Query cached = new CachingWrapperQuery(expected, MAYBE_CACHE_POLICY);
|
||||
assertQueryEquals(expected, cached);
|
||||
}
|
||||
}
|
||||
|
||||
/** test sparse filters (match single documents) */
|
||||
public void testSparse() throws Exception {
|
||||
for (int i = 0; i < 10; i++) {
|
||||
int id_start = random().nextInt(ir.maxDoc()-1);
|
||||
int id_end = id_start + 1;
|
||||
Query expected = TermRangeQuery.newStringRange("id",
|
||||
Integer.toString(id_start), Integer.toString(id_end), true, true);
|
||||
Query cached = new CachingWrapperQuery(expected, MAYBE_CACHE_POLICY);
|
||||
assertQueryEquals(expected, cached);
|
||||
}
|
||||
}
|
||||
|
||||
/** test dense filters (match entire index) */
|
||||
public void testDense() throws Exception {
|
||||
Query query = new MatchAllDocsQuery();
|
||||
Filter expected = new QueryWrapperFilter(query);
|
||||
Query cached = new CachingWrapperQuery(expected, MAYBE_CACHE_POLICY);
|
||||
assertQueryEquals(expected, cached);
|
||||
}
|
||||
|
||||
public void testCachingWorks() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
|
||||
writer.close();
|
||||
|
||||
IndexReader reader = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(dir));
|
||||
IndexSearcher searcher = newSearcher(reader);
|
||||
LeafReaderContext context = (LeafReaderContext) reader.getContext();
|
||||
MockFilter filter = new MockFilter();
|
||||
CachingWrapperQuery cacher = new CachingWrapperQuery(filter, QueryCachingPolicy.ALWAYS_CACHE);
|
||||
|
||||
// first time, nested filter is called
|
||||
cacher.createWeight(searcher, false).scorer(context, context.reader().getLiveDocs());
|
||||
assertTrue("first time", filter.wasCalled());
|
||||
|
||||
// make sure no exception if cache is holding the wrong docIdSet
|
||||
cacher.createWeight(searcher, false).scorer(context, context.reader().getLiveDocs());
|
||||
|
||||
// second time, nested filter should not be called
|
||||
filter.clear();
|
||||
cacher.createWeight(searcher, false).scorer(context, context.reader().getLiveDocs());
|
||||
assertFalse("second time", filter.wasCalled());
|
||||
|
||||
reader.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testEnforceDeletions() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter writer = new RandomIndexWriter(
|
||||
random(),
|
||||
dir,
|
||||
newIndexWriterConfig(new MockAnalyzer(random())).
|
||||
setMergeScheduler(new SerialMergeScheduler()).
|
||||
// asserts below requires no unexpected merges:
|
||||
setMergePolicy(newLogMergePolicy(10))
|
||||
);
|
||||
|
||||
// NOTE: cannot use writer.getReader because RIW (on
|
||||
// flipping a coin) may give us a newly opened reader,
|
||||
// but we use .reopen on this reader below and expect to
|
||||
// (must) get an NRT reader:
|
||||
DirectoryReader reader = DirectoryReader.open(writer.w, true);
|
||||
// same reason we don't wrap?
|
||||
IndexSearcher searcher = newSearcher(reader, false);
|
||||
|
||||
// add a doc, refresh the reader, and check that it's there
|
||||
Document doc = new Document();
|
||||
doc.add(newStringField("id", "1", Field.Store.YES));
|
||||
writer.addDocument(doc);
|
||||
|
||||
reader = refreshReader(reader);
|
||||
searcher = newSearcher(reader, false);
|
||||
|
||||
TopDocs docs = searcher.search(new MatchAllDocsQuery(), 1);
|
||||
assertEquals("Should find a hit...", 1, docs.totalHits);
|
||||
|
||||
final Query startQuery = new TermQuery(new Term("id", "1"));
|
||||
|
||||
CachingWrapperQuery query = new CachingWrapperQuery(startQuery, QueryCachingPolicy.ALWAYS_CACHE);
|
||||
|
||||
docs = searcher.search(new ConstantScoreQuery(query), 1);
|
||||
assertTrue(query.ramBytesUsed() > 0);
|
||||
|
||||
assertEquals("[query + filter] Should find a hit...", 1, docs.totalHits);
|
||||
|
||||
Query constantScore = new ConstantScoreQuery(query);
|
||||
docs = searcher.search(constantScore, 1);
|
||||
assertEquals("[just filter] Should find a hit...", 1, docs.totalHits);
|
||||
|
||||
// make sure we get a cache hit when we reopen reader
|
||||
// that had no change to deletions
|
||||
|
||||
// fake delete (deletes nothing):
|
||||
writer.deleteDocuments(new Term("foo", "bar"));
|
||||
|
||||
IndexReader oldReader = reader;
|
||||
reader = refreshReader(reader);
|
||||
assertTrue(reader == oldReader);
|
||||
int missCount = query.missCount;
|
||||
docs = searcher.search(constantScore, 1);
|
||||
assertEquals("[just filter] Should find a hit...", 1, docs.totalHits);
|
||||
|
||||
// cache hit:
|
||||
assertEquals(missCount, query.missCount);
|
||||
|
||||
// now delete the doc, refresh the reader, and see that it's not there
|
||||
writer.deleteDocuments(new Term("id", "1"));
|
||||
|
||||
// NOTE: important to hold ref here so GC doesn't clear
|
||||
// the cache entry! Else the assert below may sometimes
|
||||
// fail:
|
||||
oldReader = reader;
|
||||
reader = refreshReader(reader);
|
||||
|
||||
searcher = newSearcher(reader, false);
|
||||
|
||||
missCount = query.missCount;
|
||||
docs = searcher.search(new ConstantScoreQuery(query), 1);
|
||||
assertEquals("[query + filter] Should *not* find a hit...", 0, docs.totalHits);
|
||||
|
||||
// cache hit
|
||||
assertEquals(missCount, query.missCount);
|
||||
docs = searcher.search(constantScore, 1);
|
||||
assertEquals("[just filter] Should *not* find a hit...", 0, docs.totalHits);
|
||||
|
||||
// apply deletes dynamically:
|
||||
query = new CachingWrapperQuery(startQuery, QueryCachingPolicy.ALWAYS_CACHE);
|
||||
writer.addDocument(doc);
|
||||
reader = refreshReader(reader);
|
||||
searcher = newSearcher(reader, false);
|
||||
|
||||
docs = searcher.search(new ConstantScoreQuery(query), 1);
|
||||
assertEquals("[query + filter] Should find a hit...", 1, docs.totalHits);
|
||||
missCount = query.missCount;
|
||||
assertTrue(missCount > 0);
|
||||
constantScore = new ConstantScoreQuery(query);
|
||||
docs = searcher.search(constantScore, 1);
|
||||
assertEquals("[just filter] Should find a hit...", 1, docs.totalHits);
|
||||
assertEquals(missCount, query.missCount);
|
||||
|
||||
writer.addDocument(doc);
|
||||
|
||||
// NOTE: important to hold ref here so GC doesn't clear
|
||||
// the cache entry! Else the assert below may sometimes
|
||||
// fail:
|
||||
oldReader = reader;
|
||||
|
||||
reader = refreshReader(reader);
|
||||
searcher = newSearcher(reader, false);
|
||||
|
||||
docs = searcher.search(new ConstantScoreQuery(query), 1);
|
||||
assertEquals("[query + filter] Should find 2 hits...", 2, docs.totalHits);
|
||||
assertTrue(query.missCount > missCount);
|
||||
missCount = query.missCount;
|
||||
|
||||
constantScore = new ConstantScoreQuery(query);
|
||||
docs = searcher.search(constantScore, 1);
|
||||
assertEquals("[just filter] Should find a hit...", 2, docs.totalHits);
|
||||
assertEquals(missCount, query.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 = newSearcher(reader, false);
|
||||
|
||||
docs = searcher.search(new ConstantScoreQuery(query), 1);
|
||||
assertEquals("[query + filter] Should *not* find a hit...", 0, docs.totalHits);
|
||||
// CWF reused the same entry (it dynamically applied the deletes):
|
||||
assertEquals(missCount, query.missCount);
|
||||
|
||||
docs = searcher.search(constantScore, 1);
|
||||
assertEquals("[just filter] Should *not* find a hit...", 0, docs.totalHits);
|
||||
// CWF reused the same entry (it dynamically applied the deletes):
|
||||
assertEquals(missCount, query.missCount);
|
||||
|
||||
// NOTE: silliness to make sure JRE does not eliminate
|
||||
// our holding onto oldReader to prevent
|
||||
// CachingWrapperFilter's WeakHashMap from dropping the
|
||||
// entry:
|
||||
assertTrue(oldReader != null);
|
||||
|
||||
reader.close();
|
||||
writer.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
private static DirectoryReader refreshReader(DirectoryReader reader) throws IOException {
|
||||
DirectoryReader oldReader = reader;
|
||||
reader = DirectoryReader.openIfChanged(reader);
|
||||
if (reader != null) {
|
||||
oldReader.close();
|
||||
return reader;
|
||||
} else {
|
||||
return oldReader;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -23,10 +23,12 @@ import org.apache.lucene.document.Document;
|
|||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.similarities.DefaultSimilarity;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
/** This class only tests some basic functionality in CSQ, the main parts are mostly
|
||||
|
@ -93,6 +95,7 @@ public class TestConstantScoreQuery extends LuceneTestCase {
|
|||
writer.close();
|
||||
// we don't wrap with AssertingIndexSearcher in order to have the original scorer in setScorer.
|
||||
searcher = newSearcher(reader, true, false);
|
||||
searcher.setQueryCache(null); // to assert on scorer impl
|
||||
|
||||
// set a similarity that does not normalize our boost away
|
||||
searcher.setSimilarity(new DefaultSimilarity() {
|
||||
|
@ -127,6 +130,27 @@ public class TestConstantScoreQuery extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
// a filter for which other queries don't have special rewrite rules
|
||||
private static class FilterWrapper extends Filter {
|
||||
|
||||
private final Filter in;
|
||||
|
||||
FilterWrapper(Filter in) {
|
||||
this.in = in;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
|
||||
return in.getDocIdSet(context, acceptDocs);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString(String field) {
|
||||
return in.toString(field);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public void testConstantScoreQueryAndFilter() throws Exception {
|
||||
Directory d = newDirectory();
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), d);
|
||||
|
@ -139,13 +163,13 @@ public class TestConstantScoreQuery extends LuceneTestCase {
|
|||
IndexReader r = w.getReader();
|
||||
w.close();
|
||||
|
||||
Filter filterB = new CachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("field", "b"))));
|
||||
Filter filterB = new FilterWrapper(new QueryWrapperFilter(new TermQuery(new Term("field", "b"))));
|
||||
Query query = new ConstantScoreQuery(filterB);
|
||||
|
||||
IndexSearcher s = newSearcher(r);
|
||||
assertEquals(1, s.search(new FilteredQuery(query, filterB), 1).totalHits); // Query for field:b, Filter field:b
|
||||
|
||||
Filter filterA = new CachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("field", "a"))));
|
||||
Filter filterA = new FilterWrapper(new QueryWrapperFilter(new TermQuery(new Term("field", "a"))));
|
||||
query = new ConstantScoreQuery(filterA);
|
||||
|
||||
assertEquals(0, s.search(new FilteredQuery(query, filterB), 1).totalHits); // Query field:b, Filter field:a
|
||||
|
|
|
@ -267,12 +267,12 @@ public class TestDocValuesRangeQuery extends LuceneTestCase {
|
|||
iw.close();
|
||||
|
||||
Query q1 = DocValuesRangeQuery.newLongRange("dv1", 0L, 100L, random().nextBoolean(), random().nextBoolean());
|
||||
Weight w = searcher.createNormalizedWeight(q1, random().nextBoolean());
|
||||
Weight w = searcher.createNormalizedWeight(q1, true);
|
||||
Scorer s = w.scorer(ctx, null);
|
||||
assertNotNull(s.asTwoPhaseIterator());
|
||||
|
||||
Query q2 = DocValuesRangeQuery.newBytesRefRange("dv2", toSortableBytes(0L), toSortableBytes(100L), random().nextBoolean(), random().nextBoolean());
|
||||
w = searcher.createNormalizedWeight(q2, random().nextBoolean());
|
||||
w = searcher.createNormalizedWeight(q2, true);
|
||||
s = w.scorer(ctx, null);
|
||||
assertNotNull(s.asTwoPhaseIterator());
|
||||
|
||||
|
|
|
@ -38,11 +38,11 @@ public class TestFilterCachingPolicy extends LuceneTestCase {
|
|||
}
|
||||
final IndexReader reader = w.getReader();
|
||||
for (float minSizeRatio : new float[] {Float.MIN_VALUE, 0.01f, 0.1f, 0.9f}) {
|
||||
final FilterCachingPolicy policy = new FilterCachingPolicy.CacheOnLargeSegments(minSizeRatio);
|
||||
final QueryCachingPolicy policy = new QueryCachingPolicy.CacheOnLargeSegments(minSizeRatio);
|
||||
for (LeafReaderContext ctx : reader.leaves()) {
|
||||
final Filter filter = new QueryWrapperFilter(new TermQuery(new Term("field", "value")));
|
||||
final DocIdSet set = null;
|
||||
final boolean shouldCache = policy.shouldCache(filter, ctx, set);
|
||||
final boolean shouldCache = policy.shouldCache(filter, ctx);
|
||||
final float sizeRatio = (float) ctx.reader().maxDoc() / reader.maxDoc();
|
||||
assertEquals(sizeRatio >= minSizeRatio, shouldCache);
|
||||
}
|
||||
|
|
|
@ -302,17 +302,38 @@ public class TestFilteredQuery extends LuceneTestCase {
|
|||
tChainedFilters(false);
|
||||
}
|
||||
|
||||
// a filter for which other queries don't have special rewrite rules
|
||||
private static class FilterWrapper extends Filter {
|
||||
|
||||
private final Filter in;
|
||||
|
||||
FilterWrapper(Filter in) {
|
||||
this.in = in;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
|
||||
return in.getDocIdSet(context, acceptDocs);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString(String field) {
|
||||
return in.toString(field);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private void tChainedFilters(final boolean useRandomAccess) throws Exception {
|
||||
Query query = new FilteredQuery(new FilteredQuery(
|
||||
new MatchAllDocsQuery(), new CachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("field", "three")))), randomFilterStrategy(random(), useRandomAccess)),
|
||||
new CachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("field", "four")))), randomFilterStrategy(random(), useRandomAccess));
|
||||
new MatchAllDocsQuery(), new FilterWrapper(new QueryWrapperFilter(new TermQuery(new Term("field", "three")))), randomFilterStrategy(random(), useRandomAccess)),
|
||||
new FilterWrapper(new QueryWrapperFilter(new TermQuery(new Term("field", "four")))), randomFilterStrategy(random(), useRandomAccess));
|
||||
ScoreDoc[] hits = searcher.search(query, 10).scoreDocs;
|
||||
assertEquals(2, hits.length);
|
||||
QueryUtils.check(random(), query, searcher);
|
||||
|
||||
// one more:
|
||||
query = new FilteredQuery(query,
|
||||
new CachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("field", "five")))), randomFilterStrategy(random(), useRandomAccess));
|
||||
new FilterWrapper(new QueryWrapperFilter(new TermQuery(new Term("field", "five")))), randomFilterStrategy(random(), useRandomAccess));
|
||||
hits = searcher.search(query, 10).scoreDocs;
|
||||
assertEquals(1, hits.length);
|
||||
QueryUtils.check(random(), query, searcher);
|
||||
|
@ -390,8 +411,8 @@ public class TestFilteredQuery extends LuceneTestCase {
|
|||
}
|
||||
|
||||
public void testRewrite() throws Exception {
|
||||
assertRewrite(new FilteredQuery(new TermQuery(new Term("field", "one")), new CachingWrapperFilter(new QueryWrapperFilter(new PrefixQuery(new Term("field", "o")))), randomFilterStrategy()), FilteredQuery.class);
|
||||
assertRewrite(new FilteredQuery(new PrefixQuery(new Term("field", "one")), new CachingWrapperFilter(new QueryWrapperFilter(new PrefixQuery(new Term("field", "o")))), randomFilterStrategy()), FilteredQuery.class);
|
||||
assertRewrite(new FilteredQuery(new TermQuery(new Term("field", "one")), new FilterWrapper(new QueryWrapperFilter(new PrefixQuery(new Term("field", "o")))), randomFilterStrategy()), FilteredQuery.class);
|
||||
assertRewrite(new FilteredQuery(new PrefixQuery(new Term("field", "one")), new FilterWrapper(new QueryWrapperFilter(new PrefixQuery(new Term("field", "o")))), randomFilterStrategy()), FilteredQuery.class);
|
||||
}
|
||||
|
||||
public void testGetFilterStrategy() {
|
||||
|
|
|
@ -23,6 +23,7 @@ import java.util.Arrays;
|
|||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -34,11 +35,13 @@ import org.apache.lucene.document.Document;
|
|||
import org.apache.lucene.document.Field.Store;
|
||||
import org.apache.lucene.document.StringField;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.SerialMergeScheduler;
|
||||
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.LuceneTestCase;
|
||||
|
@ -46,34 +49,44 @@ import org.apache.lucene.util.RamUsageTester;
|
|||
|
||||
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
|
||||
|
||||
public class TestLRUFilterCache extends LuceneTestCase {
|
||||
public class TestLRUQueryCache extends LuceneTestCase {
|
||||
|
||||
private static final FilterCachingPolicy NEVER_CACHE = new FilterCachingPolicy() {
|
||||
private static final QueryCachingPolicy NEVER_CACHE = new QueryCachingPolicy() {
|
||||
|
||||
@Override
|
||||
public void onUse(Filter filter) {}
|
||||
public void onUse(Query query) {}
|
||||
|
||||
@Override
|
||||
public boolean shouldCache(Filter filter, LeafReaderContext context, DocIdSet set) throws IOException {
|
||||
public boolean shouldCache(Query query, LeafReaderContext context) throws IOException {
|
||||
return false;
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
public void testFilterRamBytesUsed() {
|
||||
final Filter simpleFilter = new QueryWrapperFilter(new TermQuery(new Term("some_field", "some_term")));
|
||||
final long actualRamBytesUsed = RamUsageTester.sizeOf(simpleFilter);
|
||||
final long ramBytesUsed = LRUFilterCache.FILTER_DEFAULT_RAM_BYTES_USED;
|
||||
final Query simpleQuery = new TermQuery(new Term("some_field", "some_term"));
|
||||
final long actualRamBytesUsed = RamUsageTester.sizeOf(simpleQuery);
|
||||
final long ramBytesUsed = LRUQueryCache.QUERY_DEFAULT_RAM_BYTES_USED;
|
||||
// we cannot assert exactly that the constant is correct since actual
|
||||
// memory usage depends on JVM implementations and settings (eg. UseCompressedOops)
|
||||
assertEquals(actualRamBytesUsed, ramBytesUsed, actualRamBytesUsed / 2);
|
||||
}
|
||||
|
||||
public void testConcurrency() throws Throwable {
|
||||
final LRUFilterCache filterCache = new LRUFilterCache(1 + random().nextInt(20), 1 + random().nextInt(10000));
|
||||
final LRUQueryCache queryCache = new LRUQueryCache(1 + random().nextInt(20), 1 + random().nextInt(10000));
|
||||
Directory dir = newDirectory();
|
||||
final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
|
||||
SearcherManager mgr = new SearcherManager(w.w, random().nextBoolean(), new SearcherFactory());
|
||||
final SearcherFactory searcherFactory = new SearcherFactory() {
|
||||
@Override
|
||||
public IndexSearcher newSearcher(IndexReader reader) throws IOException {
|
||||
IndexSearcher searcher = new IndexSearcher(reader);
|
||||
searcher.setQueryCachingPolicy(MAYBE_CACHE_POLICY);
|
||||
searcher.setQueryCache(queryCache);
|
||||
return searcher;
|
||||
}
|
||||
};
|
||||
final boolean applyDeletes = random().nextBoolean();
|
||||
final SearcherManager mgr = new SearcherManager(w.w, applyDeletes, searcherFactory);
|
||||
final AtomicBoolean indexing = new AtomicBoolean(true);
|
||||
final AtomicReference<Throwable> error = new AtomicReference<>();
|
||||
final int numDocs = atLeast(10000);
|
||||
|
@ -90,7 +103,7 @@ public class TestLRUFilterCache extends LuceneTestCase {
|
|||
if ((i & 63) == 0) {
|
||||
mgr.maybeRefresh();
|
||||
if (rarely()) {
|
||||
filterCache.clear();
|
||||
queryCache.clear();
|
||||
}
|
||||
if (rarely()) {
|
||||
final String color = RandomPicks.randomFrom(random(), new String[] {"blue", "red", "yellow"});
|
||||
|
@ -114,13 +127,12 @@ public class TestLRUFilterCache extends LuceneTestCase {
|
|||
final IndexSearcher searcher = mgr.acquire();
|
||||
try {
|
||||
final String value = RandomPicks.randomFrom(random(), new String[] {"blue", "red", "yellow", "green"});
|
||||
final Filter f = new QueryWrapperFilter(new TermQuery(new Term("color", value)));
|
||||
final Filter cached = filterCache.doCache(f, MAYBE_CACHE_POLICY);
|
||||
final Query q = new TermQuery(new Term("color", value));
|
||||
TotalHitCountCollector collector = new TotalHitCountCollector();
|
||||
searcher.search(new ConstantScoreQuery(cached), collector);
|
||||
TotalHitCountCollector collector2 = new TotalHitCountCollector();
|
||||
searcher.search(new ConstantScoreQuery(f), collector2);
|
||||
assertEquals(collector.getTotalHits(), collector2.getTotalHits());
|
||||
searcher.search(q, collector); // will use the cache
|
||||
final int totalHits1 = collector.getTotalHits();
|
||||
final int totalHits2 = searcher.search(q, 1).totalHits; // will not use the cache because of scores
|
||||
assertEquals(totalHits2, totalHits1);
|
||||
} finally {
|
||||
mgr.release(searcher);
|
||||
}
|
||||
|
@ -143,11 +155,11 @@ public class TestLRUFilterCache extends LuceneTestCase {
|
|||
if (error.get() != null) {
|
||||
throw error.get();
|
||||
}
|
||||
filterCache.assertConsistent();
|
||||
queryCache.assertConsistent();
|
||||
mgr.close();
|
||||
w.close();
|
||||
dir.close();
|
||||
filterCache.assertConsistent();
|
||||
queryCache.assertConsistent();
|
||||
}
|
||||
|
||||
public void testLRUEviction() throws Exception {
|
||||
|
@ -164,72 +176,42 @@ public class TestLRUFilterCache extends LuceneTestCase {
|
|||
w.addDocument(doc);
|
||||
final DirectoryReader reader = w.getReader();
|
||||
final IndexSearcher searcher = newSearcher(reader);
|
||||
final LRUFilterCache filterCache = new LRUFilterCache(2, 100000);
|
||||
final LRUQueryCache queryCache = new LRUQueryCache(2, 100000);
|
||||
|
||||
final Filter blue = new QueryWrapperFilter(new TermQuery(new Term("color", "blue")));
|
||||
final Filter red = new QueryWrapperFilter(new TermQuery(new Term("color", "red")));
|
||||
final Filter green = new QueryWrapperFilter(new TermQuery(new Term("color", "green")));
|
||||
final Query blue = new TermQuery(new Term("color", "blue"));
|
||||
final Query red = new TermQuery(new Term("color", "red"));
|
||||
final Query green = new TermQuery(new Term("color", "green"));
|
||||
|
||||
assertEquals(Collections.emptyList(), filterCache.cachedFilters());
|
||||
assertEquals(Collections.emptyList(), queryCache.cachedQueries());
|
||||
|
||||
searcher.setQueryCache(queryCache);
|
||||
// the filter is not cached on any segment: no changes
|
||||
searcher.search(new ConstantScoreQuery(filterCache.doCache(green, NEVER_CACHE)), 1);
|
||||
assertEquals(Collections.emptyList(), filterCache.cachedFilters());
|
||||
searcher.setQueryCachingPolicy(NEVER_CACHE);
|
||||
searcher.search(new ConstantScoreQuery(green), 1);
|
||||
assertEquals(Collections.emptyList(), queryCache.cachedQueries());
|
||||
|
||||
searcher.search(new ConstantScoreQuery(filterCache.doCache(red, FilterCachingPolicy.ALWAYS_CACHE)), 1);
|
||||
assertEquals(Collections.singletonList(red), filterCache.cachedFilters());
|
||||
searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE);
|
||||
searcher.search(new ConstantScoreQuery(red), 1);
|
||||
assertEquals(Collections.singletonList(red), queryCache.cachedQueries());
|
||||
|
||||
searcher.search(new ConstantScoreQuery(filterCache.doCache(green, FilterCachingPolicy.ALWAYS_CACHE)), 1);
|
||||
assertEquals(Arrays.asList(red, green), filterCache.cachedFilters());
|
||||
searcher.search(new ConstantScoreQuery(green), 1);
|
||||
assertEquals(Arrays.asList(red, green), queryCache.cachedQueries());
|
||||
|
||||
searcher.search(new ConstantScoreQuery(filterCache.doCache(red, FilterCachingPolicy.ALWAYS_CACHE)), 1);
|
||||
assertEquals(Arrays.asList(green, red), filterCache.cachedFilters());
|
||||
searcher.search(new ConstantScoreQuery(red), 1);
|
||||
assertEquals(Arrays.asList(green, red), queryCache.cachedQueries());
|
||||
|
||||
searcher.search(new ConstantScoreQuery(filterCache.doCache(blue, FilterCachingPolicy.ALWAYS_CACHE)), 1);
|
||||
assertEquals(Arrays.asList(red, blue), filterCache.cachedFilters());
|
||||
searcher.search(new ConstantScoreQuery(blue), 1);
|
||||
assertEquals(Arrays.asList(red, blue), queryCache.cachedQueries());
|
||||
|
||||
searcher.search(new ConstantScoreQuery(filterCache.doCache(blue, FilterCachingPolicy.ALWAYS_CACHE)), 1);
|
||||
assertEquals(Arrays.asList(red, blue), filterCache.cachedFilters());
|
||||
searcher.search(new ConstantScoreQuery(blue), 1);
|
||||
assertEquals(Arrays.asList(red, blue), queryCache.cachedQueries());
|
||||
|
||||
searcher.search(new ConstantScoreQuery(filterCache.doCache(green, FilterCachingPolicy.ALWAYS_CACHE)), 1);
|
||||
assertEquals(Arrays.asList(blue, green), filterCache.cachedFilters());
|
||||
searcher.search(new ConstantScoreQuery(green), 1);
|
||||
assertEquals(Arrays.asList(blue, green), queryCache.cachedQueries());
|
||||
|
||||
searcher.search(new ConstantScoreQuery(filterCache.doCache(red, NEVER_CACHE)), 1);
|
||||
assertEquals(Arrays.asList(blue, green), filterCache.cachedFilters());
|
||||
|
||||
reader.close();
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testCache() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
|
||||
|
||||
Document doc = new Document();
|
||||
StringField f = new StringField("color", "", Store.NO);
|
||||
doc.add(f);
|
||||
final int numDocs = atLeast(10);
|
||||
for (int i = 0; i < numDocs; ++i) {
|
||||
f.setStringValue(RandomPicks.randomFrom(random(), Arrays.asList("blue", "red", "green")));
|
||||
w.addDocument(doc);
|
||||
}
|
||||
final DirectoryReader reader = w.getReader();
|
||||
final LeafReaderContext leaf1 = reader.leaves().get(0);
|
||||
|
||||
Filter filter1 = new QueryWrapperFilter(new TermQuery(new Term("color", "blue")));
|
||||
// different instance yet equal
|
||||
Filter filter2 = new QueryWrapperFilter(new TermQuery(new Term("color", "blue")));
|
||||
|
||||
final LRUFilterCache filterCache = new LRUFilterCache(Integer.MAX_VALUE, Long.MAX_VALUE);
|
||||
final Filter cachedFilter1 = filterCache.doCache(filter1, FilterCachingPolicy.ALWAYS_CACHE);
|
||||
DocIdSet cached1 = cachedFilter1.getDocIdSet(leaf1, null);
|
||||
|
||||
final Filter cachedFilter2 = filterCache.doCache(filter2, NEVER_CACHE);
|
||||
DocIdSet cached2 = cachedFilter2.getDocIdSet(leaf1, null);
|
||||
assertSame(cached1, cached2);
|
||||
|
||||
filterCache.assertConsistent();
|
||||
searcher.setQueryCachingPolicy(NEVER_CACHE);
|
||||
searcher.search(new ConstantScoreQuery(red), 1);
|
||||
assertEquals(Arrays.asList(blue, green), queryCache.cachedQueries());
|
||||
|
||||
reader.close();
|
||||
w.close();
|
||||
|
@ -249,31 +231,35 @@ public class TestLRUFilterCache extends LuceneTestCase {
|
|||
w.addDocument(doc);
|
||||
}
|
||||
final DirectoryReader reader = w.getReader();
|
||||
final LeafReaderContext leaf1 = reader.leaves().get(0);
|
||||
final IndexSearcher searcher = newSearcher(reader);
|
||||
|
||||
final Filter filter1 = new QueryWrapperFilter(new TermQuery(new Term("color", "blue")));
|
||||
final Query query1 = new TermQuery(new Term("color", "blue"));
|
||||
query1.setBoost(random().nextFloat());
|
||||
// different instance yet equal
|
||||
final Filter filter2 = new QueryWrapperFilter(new TermQuery(new Term("color", "blue")));
|
||||
final Query query2 = new TermQuery(new Term("color", "blue"));
|
||||
query2.setBoost(random().nextFloat());
|
||||
|
||||
final LRUFilterCache filterCache = new LRUFilterCache(Integer.MAX_VALUE, Long.MAX_VALUE);
|
||||
final LRUQueryCache queryCache = new LRUQueryCache(Integer.MAX_VALUE, Long.MAX_VALUE);
|
||||
searcher.setQueryCache(queryCache);
|
||||
searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE);
|
||||
|
||||
final Filter cachedFilter1 = filterCache.doCache(filter1, FilterCachingPolicy.ALWAYS_CACHE);
|
||||
cachedFilter1.getDocIdSet(leaf1, null);
|
||||
searcher.search(new ConstantScoreQuery(query1), 1);
|
||||
assertEquals(1, queryCache.cachedQueries().size());
|
||||
|
||||
filterCache.clearFilter(filter2);
|
||||
queryCache.clearQuery(query2);
|
||||
|
||||
assertTrue(filterCache.cachedFilters().isEmpty());
|
||||
filterCache.assertConsistent();
|
||||
assertTrue(queryCache.cachedQueries().isEmpty());
|
||||
queryCache.assertConsistent();
|
||||
|
||||
reader.close();
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
// This test makes sure that by making the same assumptions as LRUFilterCache, RAMUsageTester
|
||||
// This test makes sure that by making the same assumptions as LRUQueryCache, RAMUsageTester
|
||||
// computes the same memory usage.
|
||||
public void testRamBytesUsedAgreesWithRamUsageTester() throws IOException {
|
||||
final LRUFilterCache filterCache = new LRUFilterCache(1 + random().nextInt(5), 1 + random().nextInt(10000));
|
||||
final LRUQueryCache queryCache = new LRUQueryCache(1 + random().nextInt(5), 1 + random().nextInt(10000));
|
||||
// an accumulator that only sums up memory usage of referenced filters and doc id sets
|
||||
final RamUsageTester.Accumulator acc = new RamUsageTester.Accumulator() {
|
||||
@Override
|
||||
|
@ -281,8 +267,8 @@ public class TestLRUFilterCache extends LuceneTestCase {
|
|||
if (o instanceof DocIdSet) {
|
||||
return ((DocIdSet) o).ramBytesUsed();
|
||||
}
|
||||
if (o instanceof Filter) {
|
||||
return filterCache.ramBytesUsed((Filter) o);
|
||||
if (o instanceof Query) {
|
||||
return queryCache.ramBytesUsed((Query) o);
|
||||
}
|
||||
if (o.getClass().getSimpleName().equals("SegmentCoreReaders")) {
|
||||
// do not take core cache keys into account
|
||||
|
@ -293,8 +279,8 @@ public class TestLRUFilterCache extends LuceneTestCase {
|
|||
queue.addAll(map.keySet());
|
||||
queue.addAll(map.values());
|
||||
final long sizePerEntry = o instanceof LinkedHashMap
|
||||
? LRUFilterCache.LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY
|
||||
: LRUFilterCache.HASHTABLE_RAM_BYTES_PER_ENTRY;
|
||||
? LRUQueryCache.LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY
|
||||
: LRUQueryCache.HASHTABLE_RAM_BYTES_PER_ENTRY;
|
||||
return sizePerEntry * map.size();
|
||||
}
|
||||
// follow links to other objects, but ignore their memory usage
|
||||
|
@ -328,31 +314,58 @@ public class TestLRUFilterCache extends LuceneTestCase {
|
|||
w.addDocument(doc);
|
||||
}
|
||||
try (final DirectoryReader reader = w.getReader()) {
|
||||
final IndexSearcher searcher = new IndexSearcher(reader);
|
||||
final IndexSearcher searcher = newSearcher(reader);
|
||||
searcher.setQueryCache(queryCache);
|
||||
searcher.setQueryCachingPolicy(MAYBE_CACHE_POLICY);
|
||||
for (int i = 0; i < 3; ++i) {
|
||||
final Filter filter = new QueryWrapperFilter(new TermQuery(new Term("color", RandomPicks.randomFrom(random(), colors))));
|
||||
searcher.search(new ConstantScoreQuery(filterCache.doCache(filter, MAYBE_CACHE_POLICY)), 1);
|
||||
final Query query = new TermQuery(new Term("color", RandomPicks.randomFrom(random(), colors)));
|
||||
searcher.search(new ConstantScoreQuery(query), 1);
|
||||
}
|
||||
}
|
||||
filterCache.assertConsistent();
|
||||
assertEquals(RamUsageTester.sizeOf(filterCache, acc), filterCache.ramBytesUsed());
|
||||
queryCache.assertConsistent();
|
||||
assertEquals(RamUsageTester.sizeOf(queryCache, acc), queryCache.ramBytesUsed());
|
||||
}
|
||||
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
/** A filter that produces empty sets. */
|
||||
private static class DummyFilter extends Filter {
|
||||
/** A query that doesn't match anything */
|
||||
private static class DummyQuery extends Query {
|
||||
|
||||
private static int COUNTER = 0;
|
||||
private final int id;
|
||||
|
||||
DummyQuery() {
|
||||
id = COUNTER++;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
|
||||
return null;
|
||||
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||
return new ConstantScoreWeight(this) {
|
||||
@Override
|
||||
Scorer scorer(LeafReaderContext context, Bits acceptDocs, float score) throws IOException {
|
||||
return null;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (obj instanceof DummyQuery == false) {
|
||||
return false;
|
||||
}
|
||||
return id == ((DummyQuery) obj).id;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return id;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString(String field) {
|
||||
return "DummyFilter";
|
||||
return "DummyQuery";
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -362,7 +375,7 @@ public class TestLRUFilterCache extends LuceneTestCase {
|
|||
// by the cache itself, not cache entries, and we want to make sure that
|
||||
// memory usage is not grossly underestimated.
|
||||
public void testRamBytesUsedConstantEntryOverhead() throws IOException {
|
||||
final LRUFilterCache filterCache = new LRUFilterCache(1000000, 10000000);
|
||||
final LRUQueryCache queryCache = new LRUQueryCache(1000000, 10000000);
|
||||
|
||||
final RamUsageTester.Accumulator acc = new RamUsageTester.Accumulator() {
|
||||
@Override
|
||||
|
@ -370,8 +383,8 @@ public class TestLRUFilterCache extends LuceneTestCase {
|
|||
if (o instanceof DocIdSet) {
|
||||
return ((DocIdSet) o).ramBytesUsed();
|
||||
}
|
||||
if (o instanceof Filter) {
|
||||
return filterCache.ramBytesUsed((Filter) o);
|
||||
if (o instanceof Query) {
|
||||
return queryCache.ramBytesUsed((Query) o);
|
||||
}
|
||||
if (o.getClass().getSimpleName().equals("SegmentCoreReaders")) {
|
||||
// do not follow references to core cache keys
|
||||
|
@ -390,16 +403,18 @@ public class TestLRUFilterCache extends LuceneTestCase {
|
|||
}
|
||||
final DirectoryReader reader = w.getReader();
|
||||
final IndexSearcher searcher = new IndexSearcher(reader);
|
||||
searcher.setQueryCache(queryCache);
|
||||
searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE);
|
||||
|
||||
final int numFilters = atLeast(1000);
|
||||
for (int i = 0; i < numFilters; ++i) {
|
||||
final Filter filter = new DummyFilter();
|
||||
final Filter cached = filterCache.doCache(filter, FilterCachingPolicy.ALWAYS_CACHE);
|
||||
searcher.search(new ConstantScoreQuery(cached), 1);
|
||||
final int numQueries = atLeast(1000);
|
||||
for (int i = 0; i < numQueries; ++i) {
|
||||
final Query query = new DummyQuery();
|
||||
searcher.search(new ConstantScoreQuery(query), 1);
|
||||
}
|
||||
assertTrue(queryCache.getCacheCount() > 0);
|
||||
|
||||
final long actualRamBytesUsed = RamUsageTester.sizeOf(filterCache, acc);
|
||||
final long expectedRamBytesUsed = filterCache.ramBytesUsed();
|
||||
final long actualRamBytesUsed = RamUsageTester.sizeOf(queryCache, acc);
|
||||
final long expectedRamBytesUsed = queryCache.ramBytesUsed();
|
||||
// error < 30%
|
||||
assertEquals(actualRamBytesUsed, expectedRamBytesUsed, 30 * actualRamBytesUsed / 100);
|
||||
|
||||
|
@ -409,7 +424,7 @@ public class TestLRUFilterCache extends LuceneTestCase {
|
|||
}
|
||||
|
||||
public void testOnUse() throws IOException {
|
||||
final LRUFilterCache filterCache = new LRUFilterCache(1 + random().nextInt(5), 1 + random().nextInt(1000));
|
||||
final LRUQueryCache queryCache = new LRUQueryCache(1 + random().nextInt(5), 1 + random().nextInt(1000));
|
||||
|
||||
Directory dir = newDirectory();
|
||||
final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
|
||||
|
@ -428,33 +443,34 @@ public class TestLRUFilterCache extends LuceneTestCase {
|
|||
final DirectoryReader reader = w.getReader();
|
||||
final IndexSearcher searcher = new IndexSearcher(reader);
|
||||
|
||||
final Map<Filter, Integer> actualCounts = new HashMap<>();
|
||||
final Map<Filter, Integer> expectedCounts = new HashMap<>();
|
||||
final Map<Query, Integer> actualCounts = new HashMap<>();
|
||||
final Map<Query, Integer> expectedCounts = new HashMap<>();
|
||||
|
||||
final FilterCachingPolicy countingPolicy = new FilterCachingPolicy() {
|
||||
final QueryCachingPolicy countingPolicy = new QueryCachingPolicy() {
|
||||
|
||||
@Override
|
||||
public boolean shouldCache(Filter filter, LeafReaderContext context, DocIdSet set) throws IOException {
|
||||
public boolean shouldCache(Query query, LeafReaderContext context) throws IOException {
|
||||
return random().nextBoolean();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onUse(Filter filter) {
|
||||
expectedCounts.put(filter, 1 + expectedCounts.getOrDefault(filter, 0));
|
||||
public void onUse(Query query) {
|
||||
expectedCounts.put(query, 1 + expectedCounts.getOrDefault(query, 0));
|
||||
}
|
||||
};
|
||||
|
||||
Filter[] filters = new Filter[10 + random().nextInt(10)];
|
||||
Filter[] cachedFilters = new Filter[filters.length];
|
||||
for (int i = 0; i < filters.length; ++i) {
|
||||
filters[i] = new QueryWrapperFilter(new TermQuery(new Term("color", RandomPicks.randomFrom(random(), Arrays.asList("red", "blue", "green", "yellow")))));
|
||||
cachedFilters[i] = filterCache.doCache(filters[i], countingPolicy);
|
||||
Query[] queries = new Query[10 + random().nextInt(10)];
|
||||
for (int i = 0; i < queries.length; ++i) {
|
||||
queries[i] = new TermQuery(new Term("color", RandomPicks.randomFrom(random(), Arrays.asList("red", "blue", "green", "yellow"))));
|
||||
queries[i].setBoost(random().nextFloat());
|
||||
}
|
||||
|
||||
searcher.setQueryCache(queryCache);
|
||||
searcher.setQueryCachingPolicy(countingPolicy);
|
||||
for (int i = 0; i < 20; ++i) {
|
||||
final int idx = random().nextInt(filters.length);
|
||||
searcher.search(new ConstantScoreQuery(cachedFilters[idx]), 1);
|
||||
actualCounts.put(filters[idx], 1 + actualCounts.getOrDefault(filters[idx], 0));
|
||||
final int idx = random().nextInt(queries.length);
|
||||
searcher.search(new ConstantScoreQuery(queries[idx]), 1);
|
||||
actualCounts.put(queries[idx], 1 + actualCounts.getOrDefault(queries[idx], 0));
|
||||
}
|
||||
|
||||
assertEquals(actualCounts, expectedCounts);
|
||||
|
@ -465,7 +481,7 @@ public class TestLRUFilterCache extends LuceneTestCase {
|
|||
}
|
||||
|
||||
public void testStats() throws IOException {
|
||||
final LRUFilterCache filterCache = new LRUFilterCache(1, 10000000);
|
||||
final LRUQueryCache queryCache = new LRUQueryCache(1, 10000000);
|
||||
|
||||
Directory dir = newDirectory();
|
||||
final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
|
||||
|
@ -486,66 +502,67 @@ public class TestLRUFilterCache extends LuceneTestCase {
|
|||
final DirectoryReader reader = w.getReader();
|
||||
final int segmentCount = reader.leaves().size();
|
||||
final IndexSearcher searcher = new IndexSearcher(reader);
|
||||
final Filter filter = new QueryWrapperFilter(new TermQuery(new Term("color", "red")));
|
||||
final Filter filter2 = new QueryWrapperFilter(new TermQuery(new Term("color", "blue")));
|
||||
final Query query = new TermQuery(new Term("color", "red"));
|
||||
final Query query2 = new TermQuery(new Term("color", "blue"));
|
||||
|
||||
searcher.setQueryCache(queryCache);
|
||||
// first pass, lookups without caching that all miss
|
||||
Filter cached = filterCache.doCache(filter, NEVER_CACHE);
|
||||
searcher.setQueryCachingPolicy(NEVER_CACHE);
|
||||
for (int i = 0; i < 10; ++i) {
|
||||
searcher.search(new ConstantScoreQuery(cached), 1);
|
||||
searcher.search(new ConstantScoreQuery(query), 1);
|
||||
}
|
||||
assertEquals(10 * segmentCount, filterCache.getTotalCount());
|
||||
assertEquals(0, filterCache.getHitCount());
|
||||
assertEquals(10 * segmentCount, filterCache.getMissCount());
|
||||
assertEquals(0, filterCache.getCacheCount());
|
||||
assertEquals(0, filterCache.getEvictionCount());
|
||||
assertEquals(0, filterCache.getCacheSize());
|
||||
assertEquals(10 * segmentCount, queryCache.getTotalCount());
|
||||
assertEquals(0, queryCache.getHitCount());
|
||||
assertEquals(10 * segmentCount, queryCache.getMissCount());
|
||||
assertEquals(0, queryCache.getCacheCount());
|
||||
assertEquals(0, queryCache.getEvictionCount());
|
||||
assertEquals(0, queryCache.getCacheSize());
|
||||
|
||||
// second pass, lookups + caching, only the first one is a miss
|
||||
cached = filterCache.doCache(filter, FilterCachingPolicy.ALWAYS_CACHE);
|
||||
searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE);
|
||||
for (int i = 0; i < 10; ++i) {
|
||||
searcher.search(new ConstantScoreQuery(cached), 1);
|
||||
searcher.search(new ConstantScoreQuery(query), 1);
|
||||
}
|
||||
assertEquals(20 * segmentCount, filterCache.getTotalCount());
|
||||
assertEquals(9 * segmentCount, filterCache.getHitCount());
|
||||
assertEquals(11 * segmentCount, filterCache.getMissCount());
|
||||
assertEquals(1 * segmentCount, filterCache.getCacheCount());
|
||||
assertEquals(0, filterCache.getEvictionCount());
|
||||
assertEquals(1 * segmentCount, filterCache.getCacheSize());
|
||||
assertEquals(20 * segmentCount, queryCache.getTotalCount());
|
||||
assertEquals(9 * segmentCount, queryCache.getHitCount());
|
||||
assertEquals(11 * segmentCount, queryCache.getMissCount());
|
||||
assertEquals(1 * segmentCount, queryCache.getCacheCount());
|
||||
assertEquals(0, queryCache.getEvictionCount());
|
||||
assertEquals(1 * segmentCount, queryCache.getCacheSize());
|
||||
|
||||
// third pass lookups without caching, we only have hits
|
||||
cached = filterCache.doCache(filter, NEVER_CACHE);
|
||||
searcher.setQueryCachingPolicy(NEVER_CACHE);
|
||||
for (int i = 0; i < 10; ++i) {
|
||||
searcher.search(new ConstantScoreQuery(cached), 1);
|
||||
searcher.search(new ConstantScoreQuery(query), 1);
|
||||
}
|
||||
assertEquals(30 * segmentCount, filterCache.getTotalCount());
|
||||
assertEquals(19 * segmentCount, filterCache.getHitCount());
|
||||
assertEquals(11 * segmentCount, filterCache.getMissCount());
|
||||
assertEquals(1 * segmentCount, filterCache.getCacheCount());
|
||||
assertEquals(0, filterCache.getEvictionCount());
|
||||
assertEquals(1 * segmentCount, filterCache.getCacheSize());
|
||||
assertEquals(30 * segmentCount, queryCache.getTotalCount());
|
||||
assertEquals(19 * segmentCount, queryCache.getHitCount());
|
||||
assertEquals(11 * segmentCount, queryCache.getMissCount());
|
||||
assertEquals(1 * segmentCount, queryCache.getCacheCount());
|
||||
assertEquals(0, queryCache.getEvictionCount());
|
||||
assertEquals(1 * segmentCount, queryCache.getCacheSize());
|
||||
|
||||
// fourth pass with a different filter which will trigger evictions since the size is 1
|
||||
cached = filterCache.doCache(filter2, FilterCachingPolicy.ALWAYS_CACHE);
|
||||
searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE);
|
||||
for (int i = 0; i < 10; ++i) {
|
||||
searcher.search(new ConstantScoreQuery(cached), 1);
|
||||
searcher.search(new ConstantScoreQuery(query2), 1);
|
||||
}
|
||||
assertEquals(40 * segmentCount, filterCache.getTotalCount());
|
||||
assertEquals(28 * segmentCount, filterCache.getHitCount());
|
||||
assertEquals(12 * segmentCount, filterCache.getMissCount());
|
||||
assertEquals(2 * segmentCount, filterCache.getCacheCount());
|
||||
assertEquals(1 * segmentCount, filterCache.getEvictionCount());
|
||||
assertEquals(1 * segmentCount, filterCache.getCacheSize());
|
||||
assertEquals(40 * segmentCount, queryCache.getTotalCount());
|
||||
assertEquals(28 * segmentCount, queryCache.getHitCount());
|
||||
assertEquals(12 * segmentCount, queryCache.getMissCount());
|
||||
assertEquals(2 * segmentCount, queryCache.getCacheCount());
|
||||
assertEquals(1 * segmentCount, queryCache.getEvictionCount());
|
||||
assertEquals(1 * segmentCount, queryCache.getCacheSize());
|
||||
|
||||
// now close, causing evictions due to the closing of segment cores
|
||||
reader.close();
|
||||
w.close();
|
||||
assertEquals(40 * segmentCount, filterCache.getTotalCount());
|
||||
assertEquals(28 * segmentCount, filterCache.getHitCount());
|
||||
assertEquals(12 * segmentCount, filterCache.getMissCount());
|
||||
assertEquals(2 * segmentCount, filterCache.getCacheCount());
|
||||
assertEquals(2 * segmentCount, filterCache.getEvictionCount());
|
||||
assertEquals(0, filterCache.getCacheSize());
|
||||
assertEquals(40 * segmentCount, queryCache.getTotalCount());
|
||||
assertEquals(28 * segmentCount, queryCache.getHitCount());
|
||||
assertEquals(12 * segmentCount, queryCache.getMissCount());
|
||||
assertEquals(2 * segmentCount, queryCache.getCacheCount());
|
||||
assertEquals(2 * segmentCount, queryCache.getEvictionCount());
|
||||
assertEquals(0, queryCache.getCacheSize());
|
||||
|
||||
dir.close();
|
||||
}
|
||||
|
@ -595,10 +612,10 @@ public class TestLRUFilterCache extends LuceneTestCase {
|
|||
final AtomicLong ramBytesUsage = new AtomicLong();
|
||||
final AtomicLong cacheSize = new AtomicLong();
|
||||
|
||||
final LRUFilterCache filterCache = new LRUFilterCache(2, 10000000) {
|
||||
final LRUQueryCache queryCache = new LRUQueryCache(2, 10000000) {
|
||||
@Override
|
||||
protected void onHit(Object readerCoreKey, Filter filter) {
|
||||
super.onHit(readerCoreKey, filter);
|
||||
protected void onHit(Object readerCoreKey, Query query) {
|
||||
super.onHit(readerCoreKey, query);
|
||||
switch(indexId.get(readerCoreKey).intValue()) {
|
||||
case 1:
|
||||
hitCount1.incrementAndGet();
|
||||
|
@ -612,8 +629,8 @@ public class TestLRUFilterCache extends LuceneTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected void onMiss(Object readerCoreKey, Filter filter) {
|
||||
super.onMiss(readerCoreKey, filter);
|
||||
protected void onMiss(Object readerCoreKey, Query query) {
|
||||
super.onMiss(readerCoreKey, query);
|
||||
switch(indexId.get(readerCoreKey).intValue()) {
|
||||
case 1:
|
||||
missCount1.incrementAndGet();
|
||||
|
@ -627,14 +644,14 @@ public class TestLRUFilterCache extends LuceneTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected void onFilterCache(Filter filter, long ramBytesUsed) {
|
||||
super.onFilterCache(filter, ramBytesUsed);
|
||||
protected void onQueryCache(Query query, long ramBytesUsed) {
|
||||
super.onQueryCache(query, ramBytesUsed);
|
||||
ramBytesUsage.addAndGet(ramBytesUsed);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void onFilterEviction(Filter filter, long ramBytesUsed) {
|
||||
super.onFilterEviction(filter, ramBytesUsed);
|
||||
protected void onQueryEviction(Query query, long ramBytesUsed) {
|
||||
super.onQueryEviction(query, ramBytesUsed);
|
||||
ramBytesUsage.addAndGet(-ramBytesUsed);
|
||||
}
|
||||
|
||||
|
@ -660,14 +677,18 @@ public class TestLRUFilterCache extends LuceneTestCase {
|
|||
}
|
||||
};
|
||||
|
||||
final Filter filter = new QueryWrapperFilter(new TermQuery(new Term("color", "red")));
|
||||
final Filter filter2 = new QueryWrapperFilter(new TermQuery(new Term("color", "blue")));
|
||||
final Filter filter3 = new QueryWrapperFilter(new TermQuery(new Term("color", "green")));
|
||||
final Query query = new TermQuery(new Term("color", "red"));
|
||||
final Query query2 = new TermQuery(new Term("color", "blue"));
|
||||
final Query query3 = new TermQuery(new Term("color", "green"));
|
||||
|
||||
for (IndexSearcher searcher : Arrays.asList(searcher1, searcher2)) {
|
||||
searcher.setQueryCache(queryCache);
|
||||
searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE);
|
||||
}
|
||||
|
||||
// search on searcher1
|
||||
Filter cached = filterCache.doCache(filter, FilterCachingPolicy.ALWAYS_CACHE);
|
||||
for (int i = 0; i < 10; ++i) {
|
||||
searcher1.search(new ConstantScoreQuery(cached), 1);
|
||||
searcher1.search(new ConstantScoreQuery(query), 1);
|
||||
}
|
||||
assertEquals(9 * segmentCount1, hitCount1.longValue());
|
||||
assertEquals(0, hitCount2.longValue());
|
||||
|
@ -675,9 +696,8 @@ public class TestLRUFilterCache extends LuceneTestCase {
|
|||
assertEquals(0, missCount2.longValue());
|
||||
|
||||
// then on searcher2
|
||||
cached = filterCache.doCache(filter2, FilterCachingPolicy.ALWAYS_CACHE);
|
||||
for (int i = 0; i < 20; ++i) {
|
||||
searcher2.search(new ConstantScoreQuery(cached), 1);
|
||||
searcher2.search(new ConstantScoreQuery(query2), 1);
|
||||
}
|
||||
assertEquals(9 * segmentCount1, hitCount1.longValue());
|
||||
assertEquals(19 * segmentCount2, hitCount2.longValue());
|
||||
|
@ -685,29 +705,28 @@ public class TestLRUFilterCache extends LuceneTestCase {
|
|||
assertEquals(segmentCount2, missCount2.longValue());
|
||||
|
||||
// now on searcher1 again to trigger evictions
|
||||
cached = filterCache.doCache(filter3, FilterCachingPolicy.ALWAYS_CACHE);
|
||||
for (int i = 0; i < 30; ++i) {
|
||||
searcher1.search(new ConstantScoreQuery(cached), 1);
|
||||
searcher1.search(new ConstantScoreQuery(query3), 1);
|
||||
}
|
||||
assertEquals(segmentCount1, filterCache.getEvictionCount());
|
||||
assertEquals(segmentCount1, queryCache.getEvictionCount());
|
||||
assertEquals(38 * segmentCount1, hitCount1.longValue());
|
||||
assertEquals(19 * segmentCount2, hitCount2.longValue());
|
||||
assertEquals(2 * segmentCount1, missCount1.longValue());
|
||||
assertEquals(segmentCount2, missCount2.longValue());
|
||||
|
||||
// check that the recomputed stats are the same as those reported by the cache
|
||||
assertEquals(filterCache.ramBytesUsed(), (segmentCount1 + segmentCount2) * LRUFilterCache.HASHTABLE_RAM_BYTES_PER_ENTRY + ramBytesUsage.longValue());
|
||||
assertEquals(filterCache.getCacheSize(), cacheSize.longValue());
|
||||
assertEquals(queryCache.ramBytesUsed(), (segmentCount1 + segmentCount2) * LRUQueryCache.HASHTABLE_RAM_BYTES_PER_ENTRY + ramBytesUsage.longValue());
|
||||
assertEquals(queryCache.getCacheSize(), cacheSize.longValue());
|
||||
|
||||
reader1.close();
|
||||
reader2.close();
|
||||
w1.close();
|
||||
w2.close();
|
||||
|
||||
assertEquals(filterCache.ramBytesUsed(), ramBytesUsage.longValue());
|
||||
assertEquals(queryCache.ramBytesUsed(), ramBytesUsage.longValue());
|
||||
assertEquals(0, cacheSize.longValue());
|
||||
|
||||
filterCache.clear();
|
||||
queryCache.clear();
|
||||
assertEquals(0, ramBytesUsage.longValue());
|
||||
assertEquals(0, cacheSize.longValue());
|
||||
|
||||
|
@ -715,4 +734,89 @@ public class TestLRUFilterCache extends LuceneTestCase {
|
|||
dir2.close();
|
||||
}
|
||||
|
||||
public void testUseRewrittenQueryAsCacheKey() throws IOException {
|
||||
final Query expectedCacheKey = new TermQuery(new Term("foo", "bar"));
|
||||
final BooleanQuery query = new BooleanQuery();
|
||||
final Query sub = expectedCacheKey.clone();
|
||||
sub.setBoost(42);
|
||||
query.add(sub, Occur.MUST);
|
||||
|
||||
final LRUQueryCache queryCache = new LRUQueryCache(1000000, 10000000);
|
||||
Directory dir = newDirectory();
|
||||
final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
|
||||
Document doc = new Document();
|
||||
doc.add(new StringField("foo", "bar", Store.YES));
|
||||
w.addDocument(doc);
|
||||
w.commit();
|
||||
final IndexReader reader = w.getReader();
|
||||
final IndexSearcher searcher = newSearcher(reader);
|
||||
w.close();
|
||||
|
||||
final QueryCachingPolicy policy = new QueryCachingPolicy() {
|
||||
|
||||
@Override
|
||||
public boolean shouldCache(Query query, LeafReaderContext context) throws IOException {
|
||||
assertEquals(expectedCacheKey, QueryCache.cacheKey(query));
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onUse(Query query) {
|
||||
assertEquals(expectedCacheKey, QueryCache.cacheKey(query));
|
||||
}
|
||||
};
|
||||
|
||||
searcher.setQueryCache(queryCache);
|
||||
searcher.setQueryCachingPolicy(policy);
|
||||
searcher.search(query, new TotalHitCountCollector());
|
||||
|
||||
reader.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testBooleanQueryCachesSubClauses() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
|
||||
Document doc = new Document();
|
||||
doc.add(new StringField("foo", "bar", Store.YES));
|
||||
w.addDocument(doc);
|
||||
w.commit();
|
||||
final IndexReader reader = w.getReader();
|
||||
final IndexSearcher searcher = newSearcher(reader);
|
||||
w.close();
|
||||
|
||||
final LRUQueryCache queryCache = new LRUQueryCache(1000000, 10000000);
|
||||
searcher.setQueryCache(queryCache);
|
||||
searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE);
|
||||
|
||||
BooleanQuery bq = new BooleanQuery();
|
||||
TermQuery should = new TermQuery(new Term("foo", "baz"));
|
||||
TermQuery must = new TermQuery(new Term("foo", "bar"));
|
||||
TermQuery filter = new TermQuery(new Term("foo", "bar"));
|
||||
TermQuery mustNot = new TermQuery(new Term("foo", "foo"));
|
||||
bq.add(should, Occur.SHOULD);
|
||||
bq.add(must, Occur.MUST);
|
||||
bq.add(filter, Occur.FILTER);
|
||||
bq.add(mustNot, Occur.MUST_NOT);
|
||||
|
||||
// same bq but with FILTER instead of MUST
|
||||
BooleanQuery bq2 = new BooleanQuery();
|
||||
bq2.add(should, Occur.SHOULD);
|
||||
bq2.add(must, Occur.FILTER);
|
||||
bq2.add(filter, Occur.FILTER);
|
||||
bq2.add(mustNot, Occur.MUST_NOT);
|
||||
|
||||
assertEquals(Collections.emptySet(), new HashSet<>(queryCache.cachedQueries()));
|
||||
searcher.search(bq, 1);
|
||||
assertEquals(new HashSet<>(Arrays.asList(filter, mustNot)), new HashSet<>(queryCache.cachedQueries()));
|
||||
|
||||
queryCache.clear();
|
||||
assertEquals(Collections.emptySet(), new HashSet<>(queryCache.cachedQueries()));
|
||||
searcher.search(new ConstantScoreQuery(bq), 1);
|
||||
assertEquals(new HashSet<>(Arrays.asList(bq2, should, must, filter, mustNot)), new HashSet<>(queryCache.cachedQueries()));
|
||||
|
||||
reader.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
}
|
|
@ -107,6 +107,15 @@ public class TestPrefixRandom extends LuceneTestCase {
|
|||
public String toString(String field) {
|
||||
return field.toString() + ":" + prefix.toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (super.equals(obj) == false) {
|
||||
return false;
|
||||
}
|
||||
final DumbPrefixQuery that = (DumbPrefixQuery) obj;
|
||||
return prefix.equals(that.prefix);
|
||||
}
|
||||
}
|
||||
|
||||
/** test a bunch of random prefixes */
|
||||
|
|
|
@ -25,15 +25,38 @@ import org.apache.lucene.document.Field;
|
|||
import org.apache.lucene.document.Field.Store;
|
||||
import org.apache.lucene.document.StringField;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
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.English;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
public class TestQueryWrapperFilter extends LuceneTestCase {
|
||||
|
||||
// a filter for which other queries don't have special rewrite rules
|
||||
private static class FilterWrapper extends Filter {
|
||||
|
||||
private final Filter in;
|
||||
|
||||
FilterWrapper(Filter in) {
|
||||
this.in = in;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
|
||||
return in.getDocIdSet(context, acceptDocs);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString(String field) {
|
||||
return in.toString(field);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public void testBasic() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
|
||||
|
@ -51,7 +74,7 @@ public class TestQueryWrapperFilter extends LuceneTestCase {
|
|||
IndexSearcher searcher = newSearcher(reader);
|
||||
TopDocs hits = searcher.search(new FilteredQuery(new MatchAllDocsQuery(), qwf), 10);
|
||||
assertEquals(1, hits.totalHits);
|
||||
hits = searcher.search(new FilteredQuery(new MatchAllDocsQuery(), new CachingWrapperFilter(qwf)), 10);
|
||||
hits = searcher.search(new FilteredQuery(new MatchAllDocsQuery(), new FilterWrapper(qwf)), 10);
|
||||
assertEquals(1, hits.totalHits);
|
||||
|
||||
// should not throw exception with complex primitive query
|
||||
|
@ -63,7 +86,7 @@ public class TestQueryWrapperFilter extends LuceneTestCase {
|
|||
|
||||
hits = searcher.search(new FilteredQuery(new MatchAllDocsQuery(), qwf), 10);
|
||||
assertEquals(1, hits.totalHits);
|
||||
hits = searcher.search(new FilteredQuery(new MatchAllDocsQuery(), new CachingWrapperFilter(qwf)), 10);
|
||||
hits = searcher.search(new FilteredQuery(new MatchAllDocsQuery(), new FilterWrapper(qwf)), 10);
|
||||
assertEquals(1, hits.totalHits);
|
||||
|
||||
// should not throw exception with non primitive Query (doesn't implement
|
||||
|
@ -72,7 +95,7 @@ public class TestQueryWrapperFilter extends LuceneTestCase {
|
|||
|
||||
hits = searcher.search(new FilteredQuery(new MatchAllDocsQuery(), qwf), 10);
|
||||
assertEquals(1, hits.totalHits);
|
||||
hits = searcher.search(new FilteredQuery(new MatchAllDocsQuery(), new CachingWrapperFilter(qwf)), 10);
|
||||
hits = searcher.search(new FilteredQuery(new MatchAllDocsQuery(), new FilterWrapper(qwf)), 10);
|
||||
assertEquals(1, hits.totalHits);
|
||||
|
||||
// test a query with no hits
|
||||
|
@ -80,7 +103,7 @@ public class TestQueryWrapperFilter extends LuceneTestCase {
|
|||
qwf = new QueryWrapperFilter(termQuery);
|
||||
hits = searcher.search(new FilteredQuery(new MatchAllDocsQuery(), qwf), 10);
|
||||
assertEquals(0, hits.totalHits);
|
||||
hits = searcher.search(new FilteredQuery(new MatchAllDocsQuery(), new CachingWrapperFilter(qwf)), 10);
|
||||
hits = searcher.search(new FilteredQuery(new MatchAllDocsQuery(), new FilterWrapper(qwf)), 10);
|
||||
assertEquals(0, hits.totalHits);
|
||||
reader.close();
|
||||
dir.close();
|
||||
|
|
|
@ -138,6 +138,15 @@ public class TestRegexpRandom2 extends LuceneTestCase {
|
|||
public String toString(String field) {
|
||||
return field.toString() + automaton.toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (super.equals(obj) == false) {
|
||||
return false;
|
||||
}
|
||||
final DumbRegexpQuery that = (DumbRegexpQuery) obj;
|
||||
return automaton.equals(that.automaton);
|
||||
}
|
||||
}
|
||||
|
||||
/** test a bunch of random regular expressions */
|
||||
|
@ -146,7 +155,7 @@ public class TestRegexpRandom2 extends LuceneTestCase {
|
|||
for (int i = 0; i < num; i++) {
|
||||
String reg = AutomatonTestUtil.randomRegexp(random());
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: regexp=" + reg);
|
||||
System.out.println("TEST: regexp='" + reg + "'");
|
||||
}
|
||||
assertSame(reg);
|
||||
}
|
||||
|
|
|
@ -19,26 +19,26 @@ package org.apache.lucene.search;
|
|||
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.RoaringDocIdSet;
|
||||
|
||||
public class TestUsageTrackingFilterCachingPolicy extends LuceneTestCase {
|
||||
|
||||
public void testCheapToCache() {
|
||||
assertTrue(UsageTrackingFilterCachingPolicy.isCheapToCache(null));
|
||||
assertTrue(UsageTrackingFilterCachingPolicy.isCheapToCache(DocIdSet.EMPTY));
|
||||
assertTrue(UsageTrackingFilterCachingPolicy.isCheapToCache(new RoaringDocIdSet.Builder(5).add(3).build()));
|
||||
assertFalse(UsageTrackingFilterCachingPolicy.isCheapToCache(new DocValuesDocIdSet(5, null) {
|
||||
@Override
|
||||
protected boolean matchDoc(int doc) {
|
||||
return false;
|
||||
}
|
||||
}));
|
||||
public void testCostlyFilter() {
|
||||
assertTrue(UsageTrackingQueryCachingPolicy.isCostly(new PrefixQuery(new Term("field", "prefix"))));
|
||||
assertTrue(UsageTrackingQueryCachingPolicy.isCostly(NumericRangeQuery.newIntRange("intField", 8, 1, 1000, true, true)));
|
||||
assertFalse(UsageTrackingQueryCachingPolicy.isCostly(new TermQuery(new Term("field", "value"))));
|
||||
}
|
||||
|
||||
public void testCostlyFilter() {
|
||||
assertTrue(UsageTrackingFilterCachingPolicy.isCostly(new QueryWrapperFilter(new PrefixQuery(new Term("field", "prefix")))));
|
||||
assertTrue(UsageTrackingFilterCachingPolicy.isCostly(new QueryWrapperFilter(NumericRangeQuery.newIntRange("intField", 8, 1, 1000, true, true))));
|
||||
assertFalse(UsageTrackingFilterCachingPolicy.isCostly(new QueryWrapperFilter(new TermQuery(new Term("field", "value")))));
|
||||
public void testBoostIgnored() {
|
||||
Query q1 = new TermQuery(new Term("foo", "bar"));
|
||||
q1.setBoost(2);
|
||||
Query q2 = q1.clone();
|
||||
q2.setBoost(3);
|
||||
Query q3 = q1.clone();
|
||||
q3.setBoost(4);
|
||||
UsageTrackingQueryCachingPolicy policy = new UsageTrackingQueryCachingPolicy();
|
||||
policy.onUse(q1);
|
||||
policy.onUse(q2);
|
||||
assertEquals(2, policy.frequency(q3));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -38,7 +38,6 @@ public class TestNotDocIdSet extends BaseDocIdSetTestCase<NotDocIdSet> {
|
|||
throws IOException {
|
||||
super.assertEquals(numBits, ds1, ds2);
|
||||
final Bits bits2 = ds2.bits();
|
||||
assertTrue(ds2.isCacheable()); // since we wrapped a FixedBitSet
|
||||
assertNotNull(bits2); // since we wrapped a FixedBitSet
|
||||
assertEquals(numBits, bits2.length());
|
||||
for (int i = 0; i < numBits; ++i) {
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetField;
|
|||
import org.apache.lucene.facet.sortedset.SortedSetDocValuesReaderState;
|
||||
import org.apache.lucene.facet.taxonomy.FastTaxonomyFacetCounts;
|
||||
import org.apache.lucene.facet.taxonomy.TaxonomyReader;
|
||||
import org.apache.lucene.search.FilterCollector;
|
||||
import org.apache.lucene.search.LeafCollector;
|
||||
import org.apache.lucene.search.BooleanClause;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
|
@ -173,6 +174,16 @@ public class DrillSideways {
|
|||
drillDownQueries[i-startClause] = clauses[i].getQuery();
|
||||
}
|
||||
DrillSidewaysQuery dsq = new DrillSidewaysQuery(baseQuery, drillDownCollector, drillSidewaysCollectors, drillDownQueries, scoreSubDocsAtOnce());
|
||||
if (hitCollector.needsScores() == false) {
|
||||
// this is a borrible hack in order to make sure IndexSearcher will not
|
||||
// attempt to cache the DrillSidewaysQuery
|
||||
hitCollector = new FilterCollector(hitCollector) {
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true;
|
||||
}
|
||||
};
|
||||
}
|
||||
searcher.search(dsq, hitCollector);
|
||||
|
||||
return new DrillSidewaysResult(buildFacetsResult(drillDownCollector, drillSidewaysCollectors, drillDownDims.keySet().toArray(new String[drillDownDims.size()])), null);
|
||||
|
|
|
@ -33,8 +33,10 @@ import org.apache.lucene.search.Scorer;
|
|||
import org.apache.lucene.search.Weight;
|
||||
import org.apache.lucene.util.Bits;
|
||||
/** Only purpose is to punch through and return a
|
||||
* DrillSidewaysScorer */
|
||||
* DrillSidewaysScorer*/
|
||||
|
||||
// TODO change the way DrillSidewaysScorer is used, this query does not work
|
||||
// with filter caching
|
||||
class DrillSidewaysQuery extends Query {
|
||||
final Query baseQuery;
|
||||
final Collector drillDownCollector;
|
||||
|
|
|
@ -55,17 +55,18 @@ import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
|
|||
import org.apache.lucene.queries.function.valuesource.DoubleFieldSource;
|
||||
import org.apache.lucene.queries.function.valuesource.FloatFieldSource;
|
||||
import org.apache.lucene.queries.function.valuesource.LongFieldSource;
|
||||
import org.apache.lucene.search.CachingWrapperFilter;
|
||||
import org.apache.lucene.search.CachingWrapperQuery;
|
||||
import org.apache.lucene.search.DocIdSet;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.Filter;
|
||||
import org.apache.lucene.search.FilterCachingPolicy;
|
||||
import org.apache.lucene.search.QueryCachingPolicy;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||
import org.apache.lucene.search.NumericRangeQuery;
|
||||
import org.apache.lucene.search.QueryWrapperFilter;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BitDocIdSet;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.FixedBitSet;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
@ -908,16 +909,19 @@ public class TestRangeFacetCounts extends FacetTestCase {
|
|||
final AtomicBoolean filterWasUsed = new AtomicBoolean();
|
||||
if (random().nextBoolean()) {
|
||||
// Sort of silly:
|
||||
fastMatchFilter = new CachingWrapperFilter(new QueryWrapperFilter(new MatchAllDocsQuery()), FilterCachingPolicy.ALWAYS_CACHE) {
|
||||
@Override
|
||||
protected DocIdSet cacheImpl(DocIdSetIterator iterator, LeafReader reader)
|
||||
final Filter in = new QueryWrapperFilter(new MatchAllDocsQuery());
|
||||
fastMatchFilter = new Filter() {
|
||||
@Override
|
||||
public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs)
|
||||
throws IOException {
|
||||
final FixedBitSet cached = new FixedBitSet(reader.maxDoc());
|
||||
filterWasUsed.set(true);
|
||||
cached.or(iterator);
|
||||
return new BitDocIdSet(cached);
|
||||
}
|
||||
};
|
||||
filterWasUsed.set(true);
|
||||
return in.getDocIdSet(context, acceptDocs);
|
||||
}
|
||||
@Override
|
||||
public String toString(String field) {
|
||||
return in.toString(field);
|
||||
}
|
||||
};
|
||||
} else {
|
||||
fastMatchFilter = null;
|
||||
}
|
||||
|
|
|
@ -28,7 +28,7 @@ import org.apache.lucene.index.RandomIndexWriter;
|
|||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.queries.function.ValueSource;
|
||||
import org.apache.lucene.queries.function.valuesource.BytesRefFieldSource;
|
||||
import org.apache.lucene.search.CachingWrapperFilter;
|
||||
import org.apache.lucene.search.CachingWrapperQuery;
|
||||
import org.apache.lucene.search.Filter;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.QueryWrapperFilter;
|
||||
|
@ -159,7 +159,7 @@ public class GroupingSearchTest extends LuceneTestCase {
|
|||
assertEquals(1, group.scoreDocs.length);
|
||||
assertEquals(6, group.scoreDocs[0].doc);
|
||||
|
||||
Filter lastDocInBlock = new CachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("groupend", "x"))));
|
||||
Filter lastDocInBlock = new QueryWrapperFilter(new TermQuery(new Term("groupend", "x")));
|
||||
groupingSearch = new GroupingSearch(lastDocInBlock);
|
||||
groups = groupingSearch.search(indexSearcher, new TermQuery(new Term("content", "random")), 0, 10);
|
||||
|
||||
|
|
|
@ -51,7 +51,7 @@ import org.apache.lucene.index.Term;
|
|||
import org.apache.lucene.queries.function.ValueSource;
|
||||
import org.apache.lucene.queries.function.valuesource.BytesRefFieldSource;
|
||||
import org.apache.lucene.search.CachingCollector;
|
||||
import org.apache.lucene.search.CachingWrapperFilter;
|
||||
import org.apache.lucene.search.CachingWrapperQuery;
|
||||
import org.apache.lucene.search.Collector;
|
||||
import org.apache.lucene.search.FieldDoc;
|
||||
import org.apache.lucene.search.Filter;
|
||||
|
@ -798,7 +798,7 @@ public class TestGrouping extends LuceneTestCase {
|
|||
// group, so we can use single pass collector
|
||||
dirBlocks = newDirectory();
|
||||
rBlocks = getDocBlockReader(dirBlocks, groupDocs);
|
||||
final Filter lastDocInBlock = new CachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("groupend", "x"))));
|
||||
final Filter lastDocInBlock = new QueryWrapperFilter(new TermQuery(new Term("groupend", "x")));
|
||||
final NumericDocValues docIDToIDBlocks = MultiDocValues.getNumericValues(rBlocks, "id");
|
||||
assertNotNull(docIDToIDBlocks);
|
||||
|
||||
|
|
|
@ -18,81 +18,82 @@ package org.apache.lucene.search.join;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import java.util.WeakHashMap;
|
||||
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.search.CachingWrapperFilter;
|
||||
import org.apache.lucene.search.DocIdSet;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.Filter;
|
||||
import org.apache.lucene.search.FilterCachingPolicy;
|
||||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.BitDocIdSet;
|
||||
|
||||
/**
|
||||
* A filter wrapper that transforms the produces doc id sets into
|
||||
* {@link BitDocIdSet}s if necessary and caches them.
|
||||
* {@link Filter} wrapper that implements {@link BitDocIdSetFilter}.
|
||||
*/
|
||||
public class BitDocIdSetCachingWrapperFilter extends BitDocIdSetFilter implements Accountable {
|
||||
public class BitDocIdSetCachingWrapperFilter extends BitDocIdSetFilter {
|
||||
private final Filter filter;
|
||||
private final Map<Object,DocIdSet> cache = Collections.synchronizedMap(new WeakHashMap<>());
|
||||
|
||||
private final CachingWrapperFilter filter;
|
||||
|
||||
/** Sole constructor. */
|
||||
/** Wraps another filter's result and caches it into bitsets.
|
||||
* @param filter Filter to cache results of
|
||||
*/
|
||||
public BitDocIdSetCachingWrapperFilter(Filter filter) {
|
||||
super();
|
||||
this.filter = new CachingWrapperFilter(filter, FilterCachingPolicy.ALWAYS_CACHE) {
|
||||
@Override
|
||||
protected BitDocIdSet docIdSetToCache(DocIdSet docIdSet, LeafReader reader) throws IOException {
|
||||
if (docIdSet == null || docIdSet instanceof BitDocIdSet) {
|
||||
// this is different from CachingWrapperFilter: even when the DocIdSet is
|
||||
// cacheable, we convert it to a BitSet since we require all the
|
||||
// cached filters to be BitSets
|
||||
return (BitDocIdSet) docIdSet;
|
||||
}
|
||||
this.filter = filter;
|
||||
}
|
||||
|
||||
final DocIdSetIterator it = docIdSet.iterator();
|
||||
if (it == null) {
|
||||
return null;
|
||||
}
|
||||
BitDocIdSet.Builder builder = new BitDocIdSet.Builder(reader.maxDoc());
|
||||
builder.or(it);
|
||||
return builder.build();
|
||||
/**
|
||||
* Gets the contained filter.
|
||||
* @return the contained filter.
|
||||
*/
|
||||
public Filter getFilter() {
|
||||
return filter;
|
||||
}
|
||||
|
||||
private BitDocIdSet docIdSetToCache(DocIdSet docIdSet, LeafReader reader) throws IOException {
|
||||
final DocIdSetIterator it = docIdSet.iterator();
|
||||
if (it == null) {
|
||||
return null;
|
||||
} else {
|
||||
BitDocIdSet.Builder builder = new BitDocIdSet.Builder(reader.maxDoc());
|
||||
builder.or(it);
|
||||
return builder.build();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public BitDocIdSet getDocIdSet(LeafReaderContext context) throws IOException {
|
||||
final LeafReader reader = context.reader();
|
||||
final Object key = reader.getCoreCacheKey();
|
||||
|
||||
DocIdSet docIdSet = cache.get(key);
|
||||
if (docIdSet == null) {
|
||||
docIdSet = filter.getDocIdSet(context, null);
|
||||
docIdSet = docIdSetToCache(docIdSet, reader);
|
||||
if (docIdSet == null) {
|
||||
// We use EMPTY as a sentinel for the empty set, which is cacheable
|
||||
docIdSet = DocIdSet.EMPTY;
|
||||
}
|
||||
};
|
||||
cache.put(key, docIdSet);
|
||||
}
|
||||
return docIdSet == DocIdSet.EMPTY ? null : (BitDocIdSet) docIdSet;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString(String field) {
|
||||
return getClass().getSimpleName() + "("+filter.toString(field)+")";
|
||||
}
|
||||
|
||||
@Override
|
||||
public BitDocIdSet getDocIdSet(LeafReaderContext context) throws IOException {
|
||||
return (BitDocIdSet) filter.getDocIdSet(context, null);
|
||||
public boolean equals(Object o) {
|
||||
if (o == null || !getClass().equals(o.getClass())) return false;
|
||||
final BitDocIdSetCachingWrapperFilter other = (BitDocIdSetCachingWrapperFilter) o;
|
||||
return this.filter.equals(other.filter);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return getClass().hashCode() ^ filter.hashCode();
|
||||
return (filter.hashCode() ^ getClass().hashCode());
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (obj instanceof BitDocIdSetCachingWrapperFilter == false) {
|
||||
return false;
|
||||
}
|
||||
return filter.equals(((BitDocIdSetCachingWrapperFilter) obj).filter);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString(String field) {
|
||||
return filter.toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return filter.ramBytesUsed();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<Accountable> getChildResources() {
|
||||
return filter.getChildResources();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -501,6 +501,8 @@ public class ToParentBlockJoinCollector implements Collector {
|
|||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return trackScores || trackMaxScore || sort.needsScores();
|
||||
// needed so that eg. BooleanQuery does not rewrite its MUST clauses to
|
||||
// FILTER since the filter scorers are hidden in Scorer.getChildren().
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,6 +17,7 @@ package org.apache.lucene.search.join;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
|
@ -27,10 +28,12 @@ import org.apache.lucene.index.DirectoryReader;
|
|||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.BooleanClause;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.search.CachingWrapperFilter;
|
||||
import org.apache.lucene.search.CachingWrapperQuery;
|
||||
import org.apache.lucene.search.DocIdSet;
|
||||
import org.apache.lucene.search.Filter;
|
||||
import org.apache.lucene.search.FilteredQuery;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
|
@ -39,6 +42,7 @@ import org.apache.lucene.search.QueryWrapperFilter;
|
|||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.search.WildcardQuery;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
|
@ -115,12 +119,33 @@ public class TestBlockJoinValidation extends LuceneTestCase {
|
|||
indexSearcher.search(blockJoinQuery, 1);
|
||||
}
|
||||
|
||||
// a filter for which other queries don't have special rewrite rules
|
||||
private static class FilterWrapper extends Filter {
|
||||
|
||||
private final Filter in;
|
||||
|
||||
FilterWrapper(Filter in) {
|
||||
this.in = in;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
|
||||
return in.getDocIdSet(context, acceptDocs);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString(String field) {
|
||||
return in.toString(field);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testValidationForToChildBjqWithChildFilterQuery() throws Exception {
|
||||
Query parentQueryWithRandomChild = createParentQuery();
|
||||
|
||||
ToChildBlockJoinQuery blockJoinQuery = new ToChildBlockJoinQuery(parentQueryWithRandomChild, parentsFilter);
|
||||
Filter childFilter = new CachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("common_field", "1"))));
|
||||
Filter childFilter = new FilterWrapper(new QueryWrapperFilter(new TermQuery(new Term("common_field", "1"))));
|
||||
thrown.expect(IllegalStateException.class);
|
||||
thrown.expectMessage(ToChildBlockJoinQuery.ILLEGAL_ADVANCE_ON_PARENT);
|
||||
indexSearcher.search(new FilteredQuery(blockJoinQuery, childFilter), 1);
|
||||
|
@ -137,8 +162,8 @@ public class TestBlockJoinValidation extends LuceneTestCase {
|
|||
// advance() method is used by ConjunctionScorer, so we need to create Boolean conjunction query
|
||||
BooleanQuery conjunctionQuery = new BooleanQuery();
|
||||
WildcardQuery childQuery = new WildcardQuery(new Term("child", createFieldValue(randomChildNumber)));
|
||||
conjunctionQuery.add(new BooleanClause(childQuery, BooleanClause.Occur.MUST));
|
||||
conjunctionQuery.add(new BooleanClause(blockJoinQuery, BooleanClause.Occur.MUST));
|
||||
conjunctionQuery.add(childQuery, BooleanClause.Occur.MUST);
|
||||
conjunctionQuery.add(blockJoinQuery, BooleanClause.Occur.MUST);
|
||||
|
||||
thrown.expect(IllegalStateException.class);
|
||||
thrown.expectMessage(ToChildBlockJoinQuery.INVALID_QUERY_MESSAGE);
|
||||
|
|
|
@ -17,9 +17,14 @@ package org.apache.lucene.index;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import static org.apache.lucene.search.DocIdSet.EMPTY;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.WeakHashMap;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.Document;
|
||||
|
@ -32,37 +37,60 @@ import org.apache.lucene.index.IndexWriterConfig;
|
|||
import org.apache.lucene.index.NumericDocValues;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.BitsFilteredDocIdSet;
|
||||
import org.apache.lucene.search.BlockJoinComparatorSource;
|
||||
import org.apache.lucene.search.CachingWrapperFilter;
|
||||
import org.apache.lucene.search.DocIdSet;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.Filter;
|
||||
import org.apache.lucene.search.FilterCachingPolicy;
|
||||
import org.apache.lucene.search.QueryWrapperFilter;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.SortField;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.BitDocIdSet;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.FixedBitSet;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.SparseFixedBitSet;
|
||||
|
||||
public class TestBlockJoinSorter extends LuceneTestCase {
|
||||
|
||||
private static class FixedBitSetCachingWrapperFilter extends CachingWrapperFilter {
|
||||
private static class BitSetCachingWrapperFilter extends Filter {
|
||||
|
||||
public FixedBitSetCachingWrapperFilter(Filter filter) {
|
||||
super(filter, FilterCachingPolicy.ALWAYS_CACHE);
|
||||
private final Filter filter;
|
||||
private final Map<Object,BitDocIdSet> cache = Collections.synchronizedMap(new WeakHashMap<Object,BitDocIdSet>());
|
||||
|
||||
public BitSetCachingWrapperFilter(Filter filter) {
|
||||
this.filter = filter;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DocIdSet cacheImpl(DocIdSetIterator iterator, LeafReader reader)
|
||||
throws IOException {
|
||||
final FixedBitSet cached = new FixedBitSet(reader.maxDoc());
|
||||
cached.or(iterator);
|
||||
return new BitDocIdSet(cached);
|
||||
public DocIdSet getDocIdSet(LeafReaderContext context, final Bits acceptDocs) throws IOException {
|
||||
final LeafReader reader = context.reader();
|
||||
final Object key = reader.getCoreCacheKey();
|
||||
|
||||
BitDocIdSet docIdSet = cache.get(key);
|
||||
if (docIdSet == null) {
|
||||
final DocIdSet uncached = filter.getDocIdSet(context, null);
|
||||
final DocIdSetIterator it = uncached == null ? null : uncached.iterator();
|
||||
if (it != null) {
|
||||
BitDocIdSet.Builder builder = new BitDocIdSet.Builder(context.reader().maxDoc());
|
||||
builder.or(it);
|
||||
docIdSet = builder.build();
|
||||
}
|
||||
if (docIdSet == null) {
|
||||
docIdSet = new BitDocIdSet(new SparseFixedBitSet(context.reader().maxDoc()));
|
||||
}
|
||||
cache.put(key, docIdSet);
|
||||
}
|
||||
|
||||
return docIdSet == EMPTY ? null : BitsFilteredDocIdSet.wrap(docIdSet, acceptDocs);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString(String field) {
|
||||
return getClass().getName() + "(" + filter.toString(field) + ")";
|
||||
}
|
||||
}
|
||||
|
||||
public void test() throws IOException {
|
||||
|
@ -92,7 +120,7 @@ public class TestBlockJoinSorter extends LuceneTestCase {
|
|||
writer.close();
|
||||
|
||||
final LeafReader reader = getOnlySegmentReader(indexReader);
|
||||
final Filter parentsFilter = new FixedBitSetCachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("parent", "true"))));
|
||||
final Filter parentsFilter = new BitSetCachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("parent", "true"))));
|
||||
final FixedBitSet parentBits = (FixedBitSet) parentsFilter.getDocIdSet(reader.getContext(), null).bits();
|
||||
final NumericDocValues parentValues = reader.getNumericDocValues("parent_val");
|
||||
final NumericDocValues childValues = reader.getNumericDocValues("child_val");
|
||||
|
|
|
@ -4,7 +4,7 @@
|
|||
package org.apache.lucene.queryparser.xml.builders;
|
||||
|
||||
import org.apache.lucene.queryparser.xml.*;
|
||||
import org.apache.lucene.search.CachingWrapperFilter;
|
||||
import org.apache.lucene.search.CachingWrapperQuery;
|
||||
import org.apache.lucene.search.Filter;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.QueryWrapperFilter;
|
||||
|
@ -48,7 +48,7 @@ public class CachedFilterBuilder implements FilterBuilder {
|
|||
private final QueryBuilderFactory queryFactory;
|
||||
private final FilterBuilderFactory filterFactory;
|
||||
|
||||
private LRUCache<Object, Filter> filterCache;
|
||||
private LRUCache<Object, Query> filterCache;
|
||||
|
||||
private final int cacheSize;
|
||||
|
||||
|
@ -81,20 +81,20 @@ public class CachedFilterBuilder implements FilterBuilder {
|
|||
f = filterFactory.getFilter(childElement);
|
||||
cacheKey = f;
|
||||
}
|
||||
Filter cachedFilter = filterCache.get(cacheKey);
|
||||
Query cachedFilter = filterCache.get(cacheKey);
|
||||
if (cachedFilter != null) {
|
||||
return cachedFilter; // cache hit
|
||||
return new QueryWrapperFilter(cachedFilter); // cache hit
|
||||
}
|
||||
|
||||
//cache miss
|
||||
if (qb != null) {
|
||||
cachedFilter = new QueryWrapperFilter(q);
|
||||
} else {
|
||||
cachedFilter = new CachingWrapperFilter(f);
|
||||
cachedFilter = new CachingWrapperQuery(f);
|
||||
}
|
||||
|
||||
filterCache.put(cacheKey, cachedFilter);
|
||||
return cachedFilter;
|
||||
return new QueryWrapperFilter(cachedFilter);
|
||||
}
|
||||
|
||||
static class LRUCache<K, V> extends java.util.LinkedHashMap<K, V> {
|
||||
|
|
|
@ -94,10 +94,9 @@ import org.apache.lucene.index.*;
|
|||
import org.apache.lucene.index.IndexReader.ReaderClosedListener;
|
||||
import org.apache.lucene.index.TermsEnum.SeekStatus;
|
||||
import org.apache.lucene.search.AssertingIndexSearcher;
|
||||
import org.apache.lucene.search.DocIdSet;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.Filter;
|
||||
import org.apache.lucene.search.FilterCachingPolicy;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.QueryCachingPolicy;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.QueryUtils.FCInvisibleMultiReader;
|
||||
import org.apache.lucene.store.BaseDirectoryWrapper;
|
||||
|
@ -442,14 +441,14 @@ public abstract class LuceneTestCase extends Assert {
|
|||
CORE_DIRECTORIES.add("RAMDirectory");
|
||||
}
|
||||
|
||||
/** A {@link org.apache.lucene.search.FilterCachingPolicy} that randomly caches. */
|
||||
public static final FilterCachingPolicy MAYBE_CACHE_POLICY = new FilterCachingPolicy() {
|
||||
/** A {@link org.apache.lucene.search.QueryCachingPolicy} that randomly caches. */
|
||||
public static final QueryCachingPolicy MAYBE_CACHE_POLICY = new QueryCachingPolicy() {
|
||||
|
||||
@Override
|
||||
public void onUse(Filter filter) {}
|
||||
public void onUse(Query query) {}
|
||||
|
||||
@Override
|
||||
public boolean shouldCache(Filter filter, LeafReaderContext context, DocIdSet set) throws IOException {
|
||||
public boolean shouldCache(Query query, LeafReaderContext context) throws IOException {
|
||||
return random().nextBoolean();
|
||||
}
|
||||
|
||||
|
@ -1701,6 +1700,7 @@ public abstract class LuceneTestCase extends Assert {
|
|||
ret = random.nextBoolean() ? new IndexSearcher(r) : new IndexSearcher(r.getContext());
|
||||
}
|
||||
ret.setSimilarity(classEnvRule.similarity);
|
||||
ret.setQueryCachingPolicy(MAYBE_CACHE_POLICY);
|
||||
return ret;
|
||||
} else {
|
||||
int threads = 0;
|
||||
|
@ -1737,6 +1737,7 @@ public abstract class LuceneTestCase extends Assert {
|
|||
: new IndexSearcher(r.getContext(), ex);
|
||||
}
|
||||
ret.setSimilarity(classEnvRule.similarity);
|
||||
ret.setQueryCachingPolicy(MAYBE_CACHE_POLICY);
|
||||
return ret;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -331,11 +331,6 @@ public class BitDocSet extends DocSetBase {
|
|||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isCacheable() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return bs.ramBytesUsed();
|
||||
|
|
|
@ -211,11 +211,6 @@ abstract class DocSetBase implements DocSet {
|
|||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isCacheable() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return bs.ramBytesUsed();
|
||||
|
|
|
@ -97,7 +97,7 @@ if (c.query instanceof TermQuery) {
|
|||
filter = (Filter)cache.get(filterQuery);
|
||||
}
|
||||
if (filter == null) { // miss
|
||||
filter = new CachingWrapperFilter(new QueryWrapperFilter(filterQuery)); // construct new entry
|
||||
filter = new QueryWrapperFilter(new CachingWrapperQuery(filterQuery)); // construct new entry
|
||||
synchronized (cache) {
|
||||
cache.put(filterQuery, filter); // cache it
|
||||
}
|
||||
|
|
|
@ -761,11 +761,6 @@ public class SortedIntDocSet extends DocSetBase {
|
|||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isCacheable() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
return RamUsageEstimator.sizeOf(docs);
|
||||
|
|
Loading…
Reference in New Issue