mirror of
https://github.com/apache/lucene.git
synced 2025-02-08 19:15:06 +00:00
LUCENE-6856: LRUQueryCache.CachingWrapperWeight should delegate the bulk scorer.
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1710832 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
453dc17567
commit
569f3155c9
@ -218,6 +218,9 @@ Bug Fixes
|
||||
source for checking for expiration (Ishan Chattopadhyaya via Mike
|
||||
McCandless)
|
||||
|
||||
* LUCENE-6856: The Weight wrapper used by LRUQueryCache now delegates to the
|
||||
original Weight's BulkScorer when applicable. (Adrien Grand)
|
||||
|
||||
Other
|
||||
|
||||
* LUCENE-6827: Use explicit capacity ArrayList instead of a LinkedList
|
||||
|
@ -30,7 +30,6 @@ import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.LeafReader.CoreClosedListener;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.ReaderUtil;
|
||||
@ -409,8 +408,20 @@ public class LRUQueryCache implements QueryCache, Accountable {
|
||||
/**
|
||||
* Default cache implementation: uses {@link RoaringDocIdSet}.
|
||||
*/
|
||||
protected DocIdSet cacheImpl(DocIdSetIterator iterator, LeafReader reader) throws IOException {
|
||||
return new RoaringDocIdSet.Builder(reader.maxDoc()).add(iterator).build();
|
||||
protected DocIdSet cacheImpl(BulkScorer scorer, int maxDoc) throws IOException {
|
||||
RoaringDocIdSet.Builder builder = new RoaringDocIdSet.Builder(maxDoc);
|
||||
scorer.score(new LeafCollector() {
|
||||
|
||||
@Override
|
||||
public void setScorer(Scorer scorer) throws IOException {}
|
||||
|
||||
@Override
|
||||
public void collect(int doc) throws IOException {
|
||||
builder.add(doc);
|
||||
}
|
||||
|
||||
}, null);
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
/**
|
||||
@ -572,6 +583,20 @@ public class LRUQueryCache implements QueryCache, Accountable {
|
||||
return worstCaseRamUsage * 5 < totalRamAvailable;
|
||||
}
|
||||
|
||||
private DocIdSet cache(LeafReaderContext context) throws IOException {
|
||||
final BulkScorer scorer = in.bulkScorer(context);
|
||||
if (scorer == null) {
|
||||
return DocIdSet.EMPTY;
|
||||
} else {
|
||||
return cacheImpl(scorer, context.reader().maxDoc());
|
||||
}
|
||||
}
|
||||
|
||||
private boolean shouldCache(LeafReaderContext context) throws IOException {
|
||||
return cacheEntryHasReasonableWorstCaseSize(ReaderUtil.getTopLevelContext(context).reader().maxDoc())
|
||||
&& policy.shouldCache(in.getQuery(), context);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(LeafReaderContext context) throws IOException {
|
||||
if (used.compareAndSet(false, true)) {
|
||||
@ -579,14 +604,8 @@ public class LRUQueryCache implements QueryCache, Accountable {
|
||||
}
|
||||
DocIdSet docIdSet = get(in.getQuery(), context);
|
||||
if (docIdSet == null) {
|
||||
if (cacheEntryHasReasonableWorstCaseSize(ReaderUtil.getTopLevelContext(context).reader().maxDoc())
|
||||
&& policy.shouldCache(in.getQuery(), context)) {
|
||||
final Scorer scorer = in.scorer(context);
|
||||
if (scorer == null) {
|
||||
docIdSet = DocIdSet.EMPTY;
|
||||
} else {
|
||||
docIdSet = cacheImpl(scorer, context.reader());
|
||||
}
|
||||
if (shouldCache(context)) {
|
||||
docIdSet = cache(context);
|
||||
putIfAbsent(in.getQuery(), context, docIdSet);
|
||||
} else {
|
||||
return in.scorer(context);
|
||||
@ -605,5 +624,32 @@ public class LRUQueryCache implements QueryCache, Accountable {
|
||||
return new ConstantScoreScorer(this, 0f, disi);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
|
||||
if (used.compareAndSet(false, true)) {
|
||||
policy.onUse(getQuery());
|
||||
}
|
||||
DocIdSet docIdSet = get(in.getQuery(), context);
|
||||
if (docIdSet == null) {
|
||||
if (shouldCache(context)) {
|
||||
docIdSet = cache(context);
|
||||
putIfAbsent(in.getQuery(), context, docIdSet);
|
||||
} else {
|
||||
return in.bulkScorer(context);
|
||||
}
|
||||
}
|
||||
|
||||
assert docIdSet != null;
|
||||
if (docIdSet == DocIdSet.EMPTY) {
|
||||
return null;
|
||||
}
|
||||
final DocIdSetIterator disi = docIdSet.iterator();
|
||||
if (disi == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
return new DefaultBulkScorer(new ConstantScoreScorer(this, 0f, disi));
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
@ -28,6 +28,7 @@ import java.util.HashSet;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
@ -810,7 +811,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
|
||||
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.build(), 1);
|
||||
assertEquals(new HashSet<>(Arrays.asList(filter, mustNot)), new HashSet<>(queryCache.cachedQueries()));
|
||||
@ -879,11 +880,11 @@ public class TestLRUQueryCache extends LuceneTestCase {
|
||||
doc.add(f);
|
||||
w.addDocument(doc);
|
||||
IndexReader reader = w.getReader();
|
||||
|
||||
|
||||
final int maxSize;
|
||||
final long maxRamBytesUsed;
|
||||
final int iters;
|
||||
|
||||
|
||||
if (TEST_NIGHTLY) {
|
||||
maxSize = TestUtil.nextInt(random(), 1, 10000);
|
||||
maxRamBytesUsed = TestUtil.nextLong(random(), 1, 5000000);
|
||||
@ -929,7 +930,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
|
||||
private static class BadQuery extends Query {
|
||||
|
||||
int[] i = new int[] {42}; // an array so that clone keeps the reference
|
||||
|
||||
|
||||
@Override
|
||||
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||
return new ConstantScoreWeight(this) {
|
||||
@ -939,17 +940,17 @@ public class TestLRUQueryCache extends LuceneTestCase {
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public String toString(String field) {
|
||||
return "BadQuery";
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return super.hashCode() ^ i[0];
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
public void testDetectMutatedQueries() throws IOException {
|
||||
@ -963,11 +964,11 @@ public class TestLRUQueryCache extends LuceneTestCase {
|
||||
final IndexSearcher searcher = newSearcher(reader);
|
||||
searcher.setQueryCache(queryCache);
|
||||
searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE);
|
||||
|
||||
|
||||
BadQuery query = new BadQuery();
|
||||
searcher.count(query);
|
||||
query.i[0] += 1; // change the hashCode!
|
||||
|
||||
|
||||
try {
|
||||
// trigger an eviction
|
||||
searcher.search(new MatchAllDocsQuery(), new TotalHitCountCollector());
|
||||
@ -980,7 +981,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
|
||||
assertTrue(cause instanceof ExecutionException);
|
||||
assertTrue(cause.getCause() instanceof ConcurrentModificationException);
|
||||
}
|
||||
|
||||
|
||||
IOUtils.close(w, reader, dir);
|
||||
}
|
||||
|
||||
@ -1081,4 +1082,85 @@ public class TestLRUQueryCache extends LuceneTestCase {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
private static class WeightWrapper extends Weight {
|
||||
|
||||
private final Weight in;
|
||||
private final AtomicBoolean scorerCalled;
|
||||
private final AtomicBoolean bulkScorerCalled;
|
||||
|
||||
protected WeightWrapper(Weight in, AtomicBoolean scorerCalled, AtomicBoolean bulkScorerCalled) {
|
||||
super(in.getQuery());
|
||||
this.in = in;
|
||||
this.scorerCalled = scorerCalled;
|
||||
this.bulkScorerCalled = bulkScorerCalled;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void extractTerms(Set<Term> terms) {
|
||||
in.extractTerms(terms);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Explanation explain(LeafReaderContext context, int doc) throws IOException {
|
||||
return in.explain(context, doc);
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getValueForNormalization() throws IOException {
|
||||
return in.getValueForNormalization();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void normalize(float norm, float boost) {
|
||||
in.normalize(norm, boost);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(LeafReaderContext context) throws IOException {
|
||||
scorerCalled.set(true);
|
||||
return in.scorer(context);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
|
||||
bulkScorerCalled.set(true);
|
||||
return in.bulkScorer(context);
|
||||
}
|
||||
}
|
||||
|
||||
public void testPropagateBulkScorer() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir);
|
||||
w.addDocument(new Document());
|
||||
IndexReader reader = w.getReader();
|
||||
w.close();
|
||||
IndexSearcher searcher = newSearcher(reader);
|
||||
LeafReaderContext leaf = searcher.getIndexReader().leaves().get(0);
|
||||
AtomicBoolean scorerCalled = new AtomicBoolean();
|
||||
AtomicBoolean bulkScorerCalled = new AtomicBoolean();
|
||||
LRUQueryCache cache = new LRUQueryCache(1, Long.MAX_VALUE);
|
||||
|
||||
// test that the bulk scorer is propagated when a scorer should not be cached
|
||||
Weight weight = searcher.createNormalizedWeight(new MatchAllDocsQuery(), false);
|
||||
weight = new WeightWrapper(weight, scorerCalled, bulkScorerCalled);
|
||||
weight = cache.doCache(weight, NEVER_CACHE);
|
||||
weight.bulkScorer(leaf);
|
||||
assertEquals(true, bulkScorerCalled.get());
|
||||
assertEquals(false, scorerCalled.get());
|
||||
assertEquals(0, cache.getCacheCount());
|
||||
|
||||
// test that the doc id set is computed using the bulk scorer
|
||||
bulkScorerCalled.set(false);
|
||||
weight = searcher.createNormalizedWeight(new MatchAllDocsQuery(), false);
|
||||
weight = new WeightWrapper(weight, scorerCalled, bulkScorerCalled);
|
||||
weight = cache.doCache(weight, QueryCachingPolicy.ALWAYS_CACHE);
|
||||
weight.scorer(leaf);
|
||||
assertEquals(true, bulkScorerCalled.get());
|
||||
assertEquals(false, scorerCalled.get());
|
||||
assertEquals(1, cache.getCacheCount());
|
||||
|
||||
searcher.getIndexReader().close();
|
||||
dir.close();
|
||||
}
|
||||
}
|
||||
|
Loading…
x
Reference in New Issue
Block a user