LUCENE-6855: Remove CachingWrapperQuery.

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1710975 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Adrien Grand 2015-10-28 10:05:43 +00:00
parent 8b396d82ce
commit 44280edf79
8 changed files with 5 additions and 700 deletions

View File

@ -74,6 +74,9 @@ API Changes
IndexOutput.getName returns its name (Dawid Weiss, Robert Muir, Mike
McCandless)
* LUCENE-6855: CachingWrapperQuery is deprecated and will be removed in 6.0.
(Adrien Grand)
Changes in Runtime Behavior
* LUCENE-6789: IndexSearcher's default Similarity is changed to BM25Similarity.

View File

@ -21,7 +21,7 @@ import java.io.IOException;
import java.util.Iterator;
import java.util.Objects;
import org.apache.lucene.search.CachingWrapperQuery;
import org.apache.lucene.search.QueryCache;
import org.apache.lucene.util.AttributeSource;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
@ -39,7 +39,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 CachingWrapperQuery} shares the same entries for this atomic reader
* {@link QueryCache} impls share 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.

View File

@ -1,198 +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 java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.WeakHashMap;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.Term;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
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, Cloneable {
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 = Objects.requireNonNull(query, "Query must not be null");
this.policy = Objects.requireNonNull(policy, "QueryCachingPolicy must not be null");
}
/** 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);
}
/**
* Gets the contained query.
* @return the contained query.
*/
public Query getQuery() {
return query;
}
/**
* 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 super.rewrite(reader);
} else {
CachingWrapperQuery clone;
try {
clone = (CachingWrapperQuery) clone();
clone.query = rewritten;
return clone;
} catch (CloneNotSupportedException e) {
throw new AssertionError(e);
}
}
}
// 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;
}
return new ConstantScoreWeight(weight.getQuery()) {
final AtomicBoolean used = new AtomicBoolean(false);
@Override
public void extractTerms(Set<Term> terms) {
weight.extractTerms(terms);
}
@Override
public Scorer scorer(LeafReaderContext context) throws IOException {
if (used.compareAndSet(false, true)) {
policy.onUse(getQuery());
}
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);
if (scorer == null) {
docIdSet = DocIdSet.EMPTY;
} else {
docIdSet = cacheImpl(scorer, context.reader());
}
cache.put(key, docIdSet);
} else {
return weight.scorer(context);
}
assert docIdSet != null;
if (docIdSet == DocIdSet.EMPTY) {
return null;
}
final DocIdSetIterator disi = docIdSet.iterator();
if (disi == null) {
return null;
}
return new ConstantScoreScorer(this, 0f, disi);
}
};
}
@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);
}
}
}

View File

@ -1,338 +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 java.util.concurrent.atomic.AtomicBoolean;
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 {
BooleanQuery.Builder expected = new BooleanQuery.Builder();
Query cached = new CachingWrapperQuery(expected.build(), MAYBE_CACHE_POLICY);
assertQueryEquals(expected.build(), cached);
}
/** test iterator returns NO_MORE_DOCS */
public void testEmpty2() throws Exception {
BooleanQuery.Builder expected = new BooleanQuery.Builder();
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.build(), MAYBE_CACHE_POLICY);
assertQueryEquals(expected.build(), 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 expected = new MatchAllDocsQuery();
Query cached = new CachingWrapperQuery(expected, MAYBE_CACHE_POLICY);
assertQueryEquals(expected, cached);
}
private static class MockQuery extends Query {
private final AtomicBoolean wasCalled = new AtomicBoolean();
public boolean wasCalled() {
return wasCalled.get();
}
public void clear() {
wasCalled.set(false);
}
@Override
public String toString(String field) {
return "Mock";
}
@Override
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
return new ConstantScoreWeight(this) {
@Override
public Scorer scorer(LeafReaderContext context) throws IOException {
wasCalled.set(true);
return new ConstantScoreScorer(this, score(), DocIdSetIterator.all(context.reader().maxDoc()));
}
};
}
}
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();
MockQuery filter = new MockQuery();
CachingWrapperQuery cacher = new CachingWrapperQuery(filter, QueryCachingPolicy.ALWAYS_CACHE);
// first time, nested filter is called
cacher.createWeight(searcher, false).scorer(context);
assertTrue("first time", filter.wasCalled());
// make sure no exception if cache is holding the wrong docIdSet
cacher.createWeight(searcher, false).scorer(context);
// second time, nested filter should not be called
filter.clear();
cacher.createWeight(searcher, false).scorer(context);
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;
}
}
}

View File

@ -37,8 +37,6 @@ public class CoreParser implements QueryBuilder {
protected Analyzer analyzer;
protected QueryParser parser;
protected QueryBuilderFactory queryFactory;
//Controls the max size of the LRU cache used for QueryFilter objects parsed.
public static int maxNumCachedQueries = 20;
/**
@ -78,7 +76,6 @@ public class CoreParser implements QueryBuilder {
queryFactory.addBuilder("UserQuery", new UserInputQueryBuilder(defaultField, analyzer));
}
queryFactory.addBuilder("ConstantScoreQuery", new ConstantScoreQueryBuilder(queryFactory));
queryFactory.addBuilder("CachedQuery", new CachedQueryBuilder(queryFactory, maxNumCachedQueries));
SpanQueryBuilderFactory sqof = new SpanQueryBuilderFactory();

View File

@ -1,103 +0,0 @@
/*
* Created on 25-Jan-2006
*/
package org.apache.lucene.queryparser.xml.builders;
import java.util.Map;
import org.apache.lucene.queryparser.xml.DOMUtils;
import org.apache.lucene.queryparser.xml.ParserException;
import org.apache.lucene.queryparser.xml.QueryBuilder;
import org.apache.lucene.queryparser.xml.QueryBuilderFactory;
import org.apache.lucene.search.CachingWrapperQuery;
import org.apache.lucene.search.Query;
import org.w3c.dom.Element;
/*
* 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.
*/
/**
* Filters are cached in an LRU Cache keyed on the contained query or filter object. Using this will
* speed up overall performance for repeated uses of the same expensive query/filter. The sorts of
* queries/filters likely to benefit from caching need not necessarily be complex - e.g. simple
* TermQuerys with a large DF (document frequency) can be expensive on large indexes.
* A good example of this might be a term query on a field with only 2 possible values -
* "true" or "false". In a large index, querying or filtering on this field requires reading
* millions of document ids from disk which can more usefully be cached as a filter bitset.
* <p>
* For Queries/Filters to be cached and reused the object must implement hashcode and
* equals methods correctly so that duplicate queries/filters can be detected in the cache.
* <p>
* The CoreParser.maxNumCachedFilters property can be used to control the size of the LRU
* Cache established during the construction of CoreParser instances.
*/
public class CachedQueryBuilder implements QueryBuilder {
private final QueryBuilderFactory queryFactory;
private LRUCache<Object, Query> queryCache;
private final int cacheSize;
public CachedQueryBuilder(QueryBuilderFactory queryFactory,
int cacheSize) {
this.queryFactory = queryFactory;
this.cacheSize = cacheSize;
}
@Override
public synchronized Query getQuery(Element e) throws ParserException {
Element childElement = DOMUtils.getFirstChildOrFail(e);
if (queryCache == null) {
queryCache = new LRUCache<>(cacheSize);
}
// Test to see if child Element is a query or filter that needs to be
// cached
QueryBuilder qb = queryFactory.getQueryBuilder(childElement.getNodeName());
Object cacheKey = null;
Query q = qb.getQuery(childElement);
cacheKey = q;
Query cachedQuery = queryCache.get(cacheKey);
if (cachedQuery != null) {
return cachedQuery; // cache hit
}
//cache miss
cachedQuery = new CachingWrapperQuery(q);
queryCache.put(cacheKey, cachedQuery);
return cachedQuery;
}
static class LRUCache<K, V> extends java.util.LinkedHashMap<K, V> {
public LRUCache(int maxsize) {
super(maxsize * 4 / 3 + 1, 0.75f, true);
this.maxsize = maxsize;
}
protected int maxsize;
@Override
protected boolean removeEldestEntry(Map.Entry<K, V> eldest) {
return size() > maxsize;
}
}
}

View File

@ -1,51 +0,0 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
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.
-->
<BooleanQuery fieldName="contents">
<Clause occurs="should">
<TermQuery>merger</TermQuery>
</Clause>
<Clause occurs="mustnot">
<TermQuery >sumitomo</TermQuery>
</Clause>
<Clause occurs="filter">
<!--
CachedFilter elements can contain any Query or Filter.
CachedFilters are cached in an LRU Cache keyed on the contained query/filter object.
Using this will speed up overall performance for repeated uses of the same expensive
query/filter. The sorts of queries likely to benefit from caching need not necessarily be
complex - e.g. simple TermQuerys with a large DF (document frequency) can be expensive
on large indexes. A good example of this might be a term query on a field with only 2 possible
values - "true" or "false". In a large index, querying or filtering on this field requires
reading millions of document ids from disk which can more usefully be cached as a
QueryFilter bitset.
For Queries/Filters to be cached and reused the object must implement hashcode and
equals methods correctly so that duplicate queries/filters can be detected in the cache.
The CoreParser.maxNumCachedFilters property can be used to control the size
of the LRU Cache established during the construction of CoreParser instances.
-->
<CachedQuery>
<!-- Example query to be cached for fast, repeated use -->
<TermQuery fieldName="contents">bank</TermQuery>
<!-- Alternatively, a filter object can be cached ....
<RangeFilter fieldName="date" lowerTerm="19870409" upperTerm="19870412"/>
-->
</CachedQuery>
</Clause>
</BooleanQuery>

View File

@ -181,11 +181,6 @@ public class TestParser extends LuceneTestCase {
dumpResults("Nested Boolean query", q, 5);
}
public void testCachedFilterXML() throws ParserException, IOException {
Query q = parse("CachedQuery.xml");
dumpResults("Cached filter", q, 5);
}
public void testNumericRangeQueryQueryXML() throws ParserException, IOException {
Query q = parse("NumericRangeQueryQuery.xml");
dumpResults("NumericRangeQuery", q, 5);