You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2015/07/06 19:49:14 UTC
svn commit: r1689470 - in /lucene/dev/branches/branch_5x: ./ lucene/
lucene/core/ lucene/core/src/java/org/apache/lucene/search/
lucene/core/src/test/org/apache/lucene/search/
Author: jpountz
Date: Mon Jul 6 17:49:13 2015
New Revision: 1689470
URL: http://svn.apache.org/r1689470
Log:
LUCENE-6639: Make LRUQueryCache consider a query as used on the first time a Scorer is pulled.
Modified:
lucene/dev/branches/branch_5x/ (props changed)
lucene/dev/branches/branch_5x/lucene/ (props changed)
lucene/dev/branches/branch_5x/lucene/CHANGES.txt (contents, props changed)
lucene/dev/branches/branch_5x/lucene/core/ (props changed)
lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/CachingWrapperQuery.java
lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
Modified: lucene/dev/branches/branch_5x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/CHANGES.txt?rev=1689470&r1=1689469&r2=1689470&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_5x/lucene/CHANGES.txt Mon Jul 6 17:49:13 2015
@@ -248,6 +248,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
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/CachingWrapperQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/CachingWrapperQuery.java?rev=1689470&r1=1689469&r2=1689470&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/CachingWrapperQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/CachingWrapperQuery.java Mon Jul 6 17:49:13 2015
@@ -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
// 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<Term> terms) {
weight.extractTerms(terms);
@@ -124,6 +128,10 @@ public class CachingWrapperQuery extends
@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();
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java?rev=1689470&r1=1689469&r2=1689470&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java Mon Jul 6 17:49:13 2015
@@ -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;
@@ -556,11 +557,15 @@ public class LRUQueryCache implements Qu
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
@@ -582,7 +587,7 @@ public class LRUQueryCache implements Qu
@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);
Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java?rev=1689470&r1=1689469&r2=1689470&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java Mon Jul 6 17:49:13 2015
@@ -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;
@@ -41,12 +42,12 @@ import org.apache.lucene.index.Directory
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;
@@ -1026,4 +1027,79 @@ public class TestLRUQueryCache extends L
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 query = new BooleanQuery();
+ 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);
+ assertEquals(1, policy.frequency(query));
+ assertEquals(1, policy.frequency(foo));
+ assertEquals(1, policy.frequency(bar));
+ }
+ }
+ }
+
+ private static class FrequencyCountingPolicy implements QueryCachingPolicy {
+ private final Map<Query,AtomicInteger> 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;
+ }
+ }
}