LUCENE-7235: Avoid taking the lock in LRUQueryCache when not necessary.

This commit is contained in:
Adrien Grand 2016-04-21 15:20:03 +02:00
parent 2ff8bb0c8b
commit 81446cf345
7 changed files with 125 additions and 149 deletions

View File

@ -70,6 +70,9 @@ Optimizations
* LUCENE-7211: Reduce memory & GC for spatial RPT Intersects when the number of
matching docs is small. (Jeff Wartes, David Smiley)
* LUCENE-7235: LRUQueryCache should not take a lock for segments that it will
not cache on anyway. (Adrien Grand)
Bug Fixes
* LUCENE-7127: Fix corner case bugs in GeoPointDistanceQuery. (Robert Muir)

View File

@ -29,11 +29,14 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.Predicate;
import org.apache.lucene.index.LeafReader.CoreClosedListener;
import org.apache.lucene.index.IndexReaderContext;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.ReaderUtil;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.TieredMergePolicy;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.RamUsageEstimator;
@ -100,6 +103,7 @@ public class LRUQueryCache implements QueryCache, Accountable {
private final int maxSize;
private final long maxRamBytesUsed;
private final Predicate<LeafReaderContext> leavesToCache;
// 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;
@ -118,18 +122,58 @@ public class LRUQueryCache implements QueryCache, Accountable {
private volatile long cacheSize;
/**
* Create a new instance that will cache at most <code>maxSize</code> queries
* with at most <code>maxRamBytesUsed</code> bytes of memory.
* Expert: Create a new instance that will cache at most <code>maxSize</code>
* queries with at most <code>maxRamBytesUsed</code> bytes of memory, only on
* leaves that satisfy {@code leavesToCache};
*/
public LRUQueryCache(int maxSize, long maxRamBytesUsed) {
public LRUQueryCache(int maxSize, long maxRamBytesUsed,
Predicate<LeafReaderContext> leavesToCache) {
this.maxSize = maxSize;
this.maxRamBytesUsed = maxRamBytesUsed;
this.leavesToCache = leavesToCache;
uniqueQueries = new LinkedHashMap<>(16, 0.75f, true);
mostRecentlyUsedQueries = uniqueQueries.keySet();
cache = new IdentityHashMap<>();
ramBytesUsed = 0;
}
/**
* Create a new instance that will cache at most <code>maxSize</code> queries
* with at most <code>maxRamBytesUsed</code> bytes of memory. Queries will
* only be cached on leaves that have more than 10k documents and have more
* than 3% of the total number of documents in the index.
* This should guarantee that all leaves from the upper
* {@link TieredMergePolicy tier} will be cached while ensuring that at most
* <tt>33</tt> leaves can make it to the cache (very likely less than 10 in
* practice), which is useful for this implementation since some operations
* perform in linear time with the number of cached leaves.
*/
public LRUQueryCache(int maxSize, long maxRamBytesUsed) {
this(maxSize, maxRamBytesUsed, new MinSegmentSizePredicate(10000, .03f));
}
// pkg-private for testing
static class MinSegmentSizePredicate implements Predicate<LeafReaderContext> {
private final int minSize;
private final float minSizeRatio;
MinSegmentSizePredicate(int minSize, float minSizeRatio) {
this.minSize = minSize;
this.minSizeRatio = minSizeRatio;
}
@Override
public boolean test(LeafReaderContext context) {
final int maxDoc = context.reader().maxDoc();
if (maxDoc < minSize) {
return false;
}
final IndexReaderContext topLevelContext = ReaderUtil.getTopLevelContext(context);
final float sizeRatio = (float) context.reader().maxDoc() / topLevelContext.reader().maxDoc();
return sizeRatio >= minSizeRatio;
}
}
/**
* Expert: callback when there is a cache hit on a given query.
* Implementing this method is typically useful in order to compute more
@ -598,9 +642,10 @@ public class LRUQueryCache implements QueryCache, Accountable {
}
}
/** Check whether this segment is eligible for caching, regardless of the query. */
private boolean shouldCache(LeafReaderContext context) throws IOException {
return cacheEntryHasReasonableWorstCaseSize(ReaderUtil.getTopLevelContext(context).reader().maxDoc())
&& policy.shouldCache(in.getQuery(), context);
&& leavesToCache.test(context);
}
@Override
@ -608,9 +653,15 @@ public class LRUQueryCache implements QueryCache, Accountable {
if (used.compareAndSet(false, true)) {
policy.onUse(getQuery());
}
// Short-circuit: Check whether this segment is eligible for caching
// before we take a lock because of #get
if (shouldCache(context) == false) {
return in.scorer(context);
}
DocIdSet docIdSet = get(in.getQuery(), context);
if (docIdSet == null) {
if (shouldCache(context)) {
if (policy.shouldCache(in.getQuery(), context)) {
docIdSet = cache(context);
putIfAbsent(in.getQuery(), context, docIdSet);
} else {
@ -635,9 +686,15 @@ public class LRUQueryCache implements QueryCache, Accountable {
if (used.compareAndSet(false, true)) {
policy.onUse(getQuery());
}
// Short-circuit: Check whether this segment is eligible for caching
// before we take a lock because of #get
if (shouldCache(context) == false) {
return in.bulkScorer(context);
}
DocIdSet docIdSet = get(in.getQuery(), context);
if (docIdSet == null) {
if (shouldCache(context)) {
if (policy.shouldCache(in.getQuery(), context)) {
docIdSet = cache(context);
putIfAbsent(in.getQuery(), context, docIdSet);
} else {

View File

@ -19,10 +19,7 @@ package org.apache.lucene.search;
import java.io.IOException;
import org.apache.lucene.index.IndexReaderContext;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.ReaderUtil;
import org.apache.lucene.index.TieredMergePolicy;
/**
* A policy defining which filters should be cached.
@ -49,54 +46,6 @@ public interface QueryCachingPolicy {
};
/** A simple policy that only caches on the largest segments of an index.
* The reasoning is that these segments likely account for most of the
* 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 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 LRUQueryCache}
* perform better when the number of cached segments is low). */
public static final CacheOnLargeSegments DEFAULT = new CacheOnLargeSegments(10000, 0.03f);
private final int minIndexSize;
private final float minSizeRatio;
/**
* Create a {@link CacheOnLargeSegments} instance that only caches on a
* given segment if the total number of documents in the index is greater
* than {@code minIndexSize} and the number of documents in the segment
* divided by the total number of documents in the index is greater than
* or equal to {@code minSizeRatio}.
*/
public CacheOnLargeSegments(int minIndexSize, float minSizeRatio) {
if (minSizeRatio <= 0 || minSizeRatio >= 1) {
throw new IllegalArgumentException("minSizeRatio must be in ]0, 1[, got " + minSizeRatio);
}
this.minIndexSize = minIndexSize;
this.minSizeRatio = minSizeRatio;
}
@Override
public void onUse(Query query) {}
@Override
public boolean shouldCache(Query query, LeafReaderContext context) throws IOException {
final IndexReaderContext topLevelContext = ReaderUtil.getTopLevelContext(context);
if (topLevelContext.reader().maxDoc() < minIndexSize) {
return false;
}
final float sizeRatio = (float) context.reader().maxDoc() / topLevelContext.reader().maxDoc();
return sizeRatio >= minSizeRatio;
}
};
/** 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. */

View File

@ -66,33 +66,20 @@ public final class UsageTrackingQueryCachingPolicy implements QueryCachingPolicy
return query instanceof TermQuery;
}
private final QueryCachingPolicy.CacheOnLargeSegments segmentPolicy;
private final FrequencyTrackingRingBuffer recentlyUsedFilters;
/**
* Create a new instance.
*
* @param minIndexSize the minimum size of the top-level index
* @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(
int minIndexSize,
float minSizeRatio,
int historySize) {
this(new QueryCachingPolicy.CacheOnLargeSegments(minIndexSize, minSizeRatio), historySize);
public UsageTrackingQueryCachingPolicy(int historySize) {
this.recentlyUsedFilters = new FrequencyTrackingRingBuffer(historySize, SENTINEL);
}
/** 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);
this(256);
}
/**
@ -160,9 +147,6 @@ public final class UsageTrackingQueryCachingPolicy implements QueryCachingPolicy
return false;
}
}
if (segmentPolicy.shouldCache(query, context) == false) {
return false;
}
final int frequency = frequency(query);
final int minFrequency = minFrequencyToCache(query);
return frequency >= minFrequency;

View File

@ -81,7 +81,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
}
public void testConcurrency() throws Throwable {
final LRUQueryCache queryCache = new LRUQueryCache(1 + random().nextInt(20), 1 + random().nextInt(10000));
final LRUQueryCache queryCache = new LRUQueryCache(1 + random().nextInt(20), 1 + random().nextInt(10000), context -> random().nextBoolean());
Directory dir = newDirectory();
final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
final SearcherFactory searcherFactory = new SearcherFactory() {
@ -184,7 +184,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
w.addDocument(doc);
final DirectoryReader reader = w.getReader();
final IndexSearcher searcher = newSearcher(reader);
final LRUQueryCache queryCache = new LRUQueryCache(2, 100000);
final LRUQueryCache queryCache = new LRUQueryCache(2, 100000, context -> true);
final Query blue = new TermQuery(new Term("color", "blue"));
final Query red = new TermQuery(new Term("color", "red"));
@ -245,7 +245,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
// different instance yet equal
final Query query2 = new TermQuery(new Term("color", "blue"));
final LRUQueryCache queryCache = new LRUQueryCache(Integer.MAX_VALUE, Long.MAX_VALUE);
final LRUQueryCache queryCache = new LRUQueryCache(Integer.MAX_VALUE, Long.MAX_VALUE, context -> true);
searcher.setQueryCache(queryCache);
searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE);
@ -265,7 +265,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
// This test makes sure that by making the same assumptions as LRUQueryCache, RAMUsageTester
// computes the same memory usage.
public void testRamBytesUsedAgreesWithRamUsageTester() throws IOException {
final LRUQueryCache queryCache = new LRUQueryCache(1 + random().nextInt(5), 1 + random().nextInt(10000));
final LRUQueryCache queryCache = new LRUQueryCache(1 + random().nextInt(5), 1 + random().nextInt(10000), context -> random().nextBoolean());
// an accumulator that only sums up memory usage of referenced filters and doc id sets
final RamUsageTester.Accumulator acc = new RamUsageTester.Accumulator() {
@Override
@ -381,7 +381,7 @@ public class TestLRUQueryCache 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 LRUQueryCache queryCache = new LRUQueryCache(1000000, 10000000);
final LRUQueryCache queryCache = new LRUQueryCache(1000000, 10000000, context -> true);
final RamUsageTester.Accumulator acc = new RamUsageTester.Accumulator() {
@Override
@ -430,7 +430,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
}
public void testOnUse() throws IOException {
final LRUQueryCache queryCache = new LRUQueryCache(1 + random().nextInt(5), 1 + random().nextInt(1000));
final LRUQueryCache queryCache = new LRUQueryCache(1 + random().nextInt(5), 1 + random().nextInt(1000), context -> random().nextBoolean());
Directory dir = newDirectory();
final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
@ -490,7 +490,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
}
public void testStats() throws IOException {
final LRUQueryCache queryCache = new LRUQueryCache(1, 10000000);
final LRUQueryCache queryCache = new LRUQueryCache(1, 10000000, context -> true);
Directory dir = newDirectory();
final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
@ -621,7 +621,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
final AtomicLong ramBytesUsage = new AtomicLong();
final AtomicLong cacheSize = new AtomicLong();
final LRUQueryCache queryCache = new LRUQueryCache(2, 10000000) {
final LRUQueryCache queryCache = new LRUQueryCache(2, 10000000, context -> true) {
@Override
protected void onHit(Object readerCoreKey, Query query) {
super.onHit(readerCoreKey, query);
@ -748,7 +748,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
final BooleanQuery.Builder query = new BooleanQuery.Builder();
query.add(new BoostQuery(expectedCacheKey, 42f), Occur.MUST);
final LRUQueryCache queryCache = new LRUQueryCache(1000000, 10000000);
final LRUQueryCache queryCache = new LRUQueryCache(1000000, 10000000, context -> random().nextBoolean());
Directory dir = newDirectory();
final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
Document doc = new Document();
@ -793,7 +793,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
final IndexSearcher searcher = newSearcher(reader);
w.close();
final LRUQueryCache queryCache = new LRUQueryCache(1000000, 10000000);
final LRUQueryCache queryCache = new LRUQueryCache(1000000, 10000000, context -> true);
searcher.setQueryCache(queryCache);
searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE);
@ -897,7 +897,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
iters = atLeast(2000);
}
final LRUQueryCache queryCache = new LRUQueryCache(maxSize, maxRamBytesUsed);
final LRUQueryCache queryCache = new LRUQueryCache(maxSize, maxRamBytesUsed, context -> random().nextBoolean());
IndexSearcher uncachedSearcher = null;
IndexSearcher cachedSearcher = null;
@ -962,7 +962,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
IndexReader reader = w.getReader();
// size of 1 so that 2nd query evicts from the cache
final LRUQueryCache queryCache = new LRUQueryCache(1, 10000);
final LRUQueryCache queryCache = new LRUQueryCache(1, 10000, context -> true);
final IndexSearcher searcher = newSearcher(reader);
searcher.setQueryCache(queryCache);
searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE);
@ -996,7 +996,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
IndexReader reader = w.getReader();
// size of 1 byte
final LRUQueryCache queryCache = new LRUQueryCache(1, 1);
final LRUQueryCache queryCache = new LRUQueryCache(1, 1, context -> random().nextBoolean());
final IndexSearcher searcher = newSearcher(reader);
searcher.setQueryCache(queryCache);
searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE);
@ -1035,7 +1035,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
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.setQueryCache(new LRUQueryCache(100, 10240, context -> random().nextBoolean()));
indexSearcher.setQueryCachingPolicy(policy);
final Query foo = new TermQuery(new Term("f", "foo"));
final Query bar = new TermQuery(new Term("f", "bar"));
@ -1141,7 +1141,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
LeafReaderContext leaf = searcher.getIndexReader().leaves().get(0);
AtomicBoolean scorerCalled = new AtomicBoolean();
AtomicBoolean bulkScorerCalled = new AtomicBoolean();
LRUQueryCache cache = new LRUQueryCache(1, Long.MAX_VALUE);
LRUQueryCache cache = new LRUQueryCache(1, Long.MAX_VALUE, context -> true);
// test that the bulk scorer is propagated when a scorer should not be cached
Weight weight = searcher.createNormalizedWeight(new MatchAllDocsQuery(), false);
@ -1172,7 +1172,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
w.addDocument(new Document());
final DirectoryReader reader = w.getReader();
final IndexSearcher searcher = newSearcher(reader);
final LRUQueryCache queryCache = new LRUQueryCache(2, 100000) {
final LRUQueryCache queryCache = new LRUQueryCache(2, 100000, context -> true) {
@Override
protected void onDocIdSetEviction(Object readerCoreKey, int numEntries, long sumRamBytesUsed) {
super.onDocIdSetEviction(readerCoreKey, numEntries, sumRamBytesUsed);
@ -1192,4 +1192,43 @@ public class TestLRUQueryCache extends LuceneTestCase {
w.close();
dir.close();
}
public void testMinSegmentSizePredicate() throws IOException {
Directory dir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig().setMergePolicy(NoMergePolicy.INSTANCE);
RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
w.addDocument(new Document());
DirectoryReader reader = w.getReader();
IndexSearcher searcher = newSearcher(reader);
searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE);
LRUQueryCache cache = new LRUQueryCache(2, 10000, new LRUQueryCache.MinSegmentSizePredicate(2, 0f));
searcher.setQueryCache(cache);
searcher.count(new DummyQuery());
assertEquals(0, cache.getCacheCount());
cache = new LRUQueryCache(2, 10000, new LRUQueryCache.MinSegmentSizePredicate(1, 0f));
searcher.setQueryCache(cache);
searcher.count(new DummyQuery());
assertEquals(1, cache.getCacheCount());
cache = new LRUQueryCache(2, 10000, new LRUQueryCache.MinSegmentSizePredicate(0, .6f));
searcher.setQueryCache(cache);
searcher.count(new DummyQuery());
assertEquals(1, cache.getCacheCount());
w.addDocument(new Document());
reader.close();
reader = w.getReader();
searcher = newSearcher(reader);
searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE);
cache = new LRUQueryCache(2, 10000, new LRUQueryCache.MinSegmentSizePredicate(0, .6f));
searcher.setQueryCache(cache);
searcher.count(new DummyQuery());
assertEquals(0, cache.getCacheCount());
reader.close();
w.close();
dir.close();
}
}

View File

@ -1,56 +0,0 @@
/*
* 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.
*/
package org.apache.lucene.search;
import java.io.IOException;
import org.apache.lucene.document.Document;
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.store.Directory;
import org.apache.lucene.util.LuceneTestCase;
public class TestQueryCachingPolicy extends LuceneTestCase {
public void testLargeSegmentDetection() throws IOException {
Directory dir = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random(), dir);
final int numDocs = atLeast(100);
for (int i = 0; i < numDocs; ++i) {
w.addDocument(new Document());
}
final IndexReader reader = w.getReader();
for (float minSizeRatio : new float[] {Float.MIN_VALUE, 0.01f, 0.1f, 0.9f}) {
final QueryCachingPolicy policy = new QueryCachingPolicy.CacheOnLargeSegments(0, minSizeRatio);
for (LeafReaderContext ctx : reader.leaves()) {
final Query query = new TermQuery(new Term("field", "value"));
final boolean shouldCache = policy.shouldCache(query, ctx);
final float sizeRatio = (float) ctx.reader().maxDoc() / reader.maxDoc();
assertEquals(sizeRatio >= minSizeRatio, shouldCache);
assertTrue(new QueryCachingPolicy.CacheOnLargeSegments(numDocs, Float.MIN_VALUE).shouldCache(query, ctx));
assertFalse(new QueryCachingPolicy.CacheOnLargeSegments(numDocs + 1, Float.MIN_VALUE).shouldCache(query, ctx));
}
}
reader.close();
w.close();
dir.close();
}
}

View File

@ -1790,7 +1790,7 @@ public abstract class LuceneTestCase extends Assert {
public static void overrideDefaultQueryCache() {
// we need to reset the query cache in an @BeforeClass so that tests that
// instantiate an IndexSearcher in an @BeforeClass method use a fresh new cache
IndexSearcher.setDefaultQueryCache(new LRUQueryCache(10000, 1 << 25));
IndexSearcher.setDefaultQueryCache(new LRUQueryCache(10000, 1 << 25, context -> true));
IndexSearcher.setDefaultQueryCachingPolicy(MAYBE_CACHE_POLICY);
}