From de18625659bcccfacfbd294fadaf1bca70dee7ac Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Mon, 6 Jul 2015 17:38:37 +0000 Subject: [PATCH] LUCENE-6639: Make LRUQueryCache consider a query as used on the first time a Scorer is pulled. git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1689464 13f79535-47bb-0310-9956-ffa450edef68 --- lucene/CHANGES.txt | 4 + .../lucene/search/CachingWrapperQuery.java | 10 ++- .../apache/lucene/search/LRUQueryCache.java | 7 +- .../lucene/search/TestLRUQueryCache.java | 78 ++++++++++++++++++- 4 files changed, 96 insertions(+), 3 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index f81e8d64f31..f742f854bdf 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -287,6 +287,10 @@ Changes in Runtime Behavior implementing this method in all your custom attributes, because the method will be made abstract in Lucene 6. (Uwe Schindler) +* LUCENE-6639: LRUQueryCache and CachingWrapperQuery now consider a query as + "used" when the first Scorer is pulled instead of when a Scorer is pulled on + the first segment on an index. (Terry Smith, Adrien Grand) + Optimizations * LUCENE-6548: Some optimizations for BlockTree's intersect with very diff --git a/lucene/core/src/java/org/apache/lucene/search/CachingWrapperQuery.java b/lucene/core/src/java/org/apache/lucene/search/CachingWrapperQuery.java index 871532ce20d..f0ae0cb90b0 100644 --- a/lucene/core/src/java/org/apache/lucene/search/CachingWrapperQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/CachingWrapperQuery.java @@ -26,6 +26,7 @@ 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; @@ -115,8 +116,11 @@ public class CachingWrapperQuery extends Query implements Accountable { // our cache is not sufficient, we need scores too return weight; } - policy.onUse(weight.getQuery()); + return new ConstantScoreWeight(weight.getQuery()) { + + final AtomicBoolean used = new AtomicBoolean(false); + @Override public void extractTerms(Set terms) { weight.extractTerms(terms); @@ -124,6 +128,10 @@ public class CachingWrapperQuery extends Query implements Accountable { @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(); diff --git a/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java b/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java index 6499501224a..8f83726b7e0 100644 --- a/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java +++ b/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java @@ -28,6 +28,7 @@ import java.util.LinkedHashMap; import java.util.List; 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; @@ -540,11 +541,15 @@ public class LRUQueryCache implements QueryCache, Accountable { private final Weight in; private final QueryCachingPolicy policy; + // we use an AtomicBoolean because Weight.scorer may be called from multiple + // threads when IndexSearcher is created with threads + private final AtomicBoolean used; CachingWrapperWeight(Weight in, QueryCachingPolicy policy) { super(in.getQuery()); this.in = in; this.policy = policy; + used = new AtomicBoolean(false); } @Override @@ -566,7 +571,7 @@ public class LRUQueryCache implements QueryCache, Accountable { @Override public Scorer scorer(LeafReaderContext context) throws IOException { - if (context.ord == 0) { + if (used.compareAndSet(false, true)) { policy.onUse(getQuery()); } DocIdSet docIdSet = get(in.getQuery(), context); diff --git a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java index 4e6e8e3b0c3..f058fb654d2 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java @@ -30,6 +30,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -43,12 +44,12 @@ 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.NoMergePolicy; 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.IOUtils; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.RamUsageTester; @@ -1006,4 +1007,79 @@ public class TestLRUQueryCache extends LuceneTestCase { w.close(); dir.close(); } + + /** + * Tests CachingWrapperWeight.scorer() propagation of {@link QueryCachingPolicy#onUse(Query)} when the first segment + * is skipped. + * + * #f:foo #f:bar causes all frequencies to increment + * #f:bar #f:foo does not increment the frequency for f:foo + */ + public void testOnUseWithRandomFirstSegmentSkipping() throws IOException { + try (final Directory directory = newDirectory()) { + try (final RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory, newIndexWriterConfig().setMergePolicy(NoMergePolicy.INSTANCE))) { + Document doc = new Document(); + doc.add(new StringField("f", "bar", Store.NO)); + indexWriter.addDocument(doc); + if (random().nextBoolean()) { + indexWriter.getReader().close(); + } + doc = new Document(); + doc.add(new StringField("f", "foo", Store.NO)); + doc.add(new StringField("f", "bar", Store.NO)); + indexWriter.addDocument(doc); + indexWriter.commit(); + } + try (final IndexReader indexReader = DirectoryReader.open(directory)) { + final FrequencyCountingPolicy policy = new FrequencyCountingPolicy(); + final IndexSearcher indexSearcher = new IndexSearcher(indexReader); + indexSearcher.setQueryCache(new LRUQueryCache(100, 10240)); + indexSearcher.setQueryCachingPolicy(policy); + final Query foo = new TermQuery(new Term("f", "foo")); + final Query bar = new TermQuery(new Term("f", "bar")); + final BooleanQuery.Builder query = new BooleanQuery.Builder(); + if (random().nextBoolean()) { + query.add(foo, Occur.FILTER); + query.add(bar, Occur.FILTER); + } else { + query.add(bar, Occur.FILTER); + query.add(foo, Occur.FILTER); + } + indexSearcher.count(query.build()); + assertEquals(1, policy.frequency(query.build())); + assertEquals(1, policy.frequency(foo)); + assertEquals(1, policy.frequency(bar)); + } + } + } + + private static class FrequencyCountingPolicy implements QueryCachingPolicy { + private final Map counts = new HashMap<>(); + + public int frequency(final Query query) { + AtomicInteger count; + synchronized (counts) { + count = counts.get(query); + } + return count != null ? count.get() : 0; + } + + @Override + public void onUse(final Query query) { + AtomicInteger count; + synchronized (counts) { + count = counts.get(query); + if (count == null) { + count = new AtomicInteger(); + counts.put(query, count); + } + } + count.incrementAndGet(); + } + + @Override + public boolean shouldCache(Query query, LeafReaderContext context) throws IOException { + return true; + } + } }