From bf232d7635e1686cd6f5624525fa3e0b7820430f Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Fri, 22 Apr 2016 14:09:44 +0200 Subject: [PATCH] LUCENE-7237: LRUQueryCache now prefers returning an uncached Scorer than waiting on a lock. --- lucene/CHANGES.txt | 3 + .../apache/lucene/search/LRUQueryCache.java | 234 +++++++++++------- 2 files changed, 154 insertions(+), 83 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 18409c99685..4b72294a1f1 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -76,6 +76,9 @@ Optimizations * LUCENE-7238: Explicitly disable the query cache in MemoryIndex#createSearcher. (Adrien Grand) +* LUCENE-7237: LRUQueryCache now prefers returning an uncached Scorer than + waiting on a lock. (Adrien Grand) + Bug Fixes * LUCENE-7127: Fix corner case bugs in GeoPointDistanceQuery. (Robert Muir) 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 15c0f2b508a..859864572b1 100644 --- a/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java +++ b/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java @@ -29,6 +29,7 @@ import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.ReentrantLock; import java.util.function.Predicate; import org.apache.lucene.index.LeafReader.CoreClosedListener; @@ -112,6 +113,7 @@ public class LRUQueryCache implements QueryCache, Accountable { // mostRecentlyUsedQueries. This is why write operations are performed under a lock private final Set mostRecentlyUsedQueries; private final Map cache; + private final ReentrantLock lock; // these variables are volatile so that we do not need to sync reads // but increments need to be performed under the lock @@ -134,6 +136,7 @@ public class LRUQueryCache implements QueryCache, Accountable { uniqueQueries = new LinkedHashMap<>(16, 0.75f, true); mostRecentlyUsedQueries = uniqueQueries.keySet(); cache = new IdentityHashMap<>(); + lock = new ReentrantLock(); ramBytesUsed = 0; } @@ -182,6 +185,7 @@ public class LRUQueryCache implements QueryCache, Accountable { * @lucene.experimental */ protected void onHit(Object readerCoreKey, Query query) { + assert lock.isHeldByCurrentThread(); hitCount += 1; } @@ -191,6 +195,7 @@ public class LRUQueryCache implements QueryCache, Accountable { * @lucene.experimental */ protected void onMiss(Object readerCoreKey, Query query) { + assert lock.isHeldByCurrentThread(); assert query != null; missCount += 1; } @@ -203,6 +208,7 @@ public class LRUQueryCache implements QueryCache, Accountable { * @lucene.experimental */ protected void onQueryCache(Query query, long ramBytesUsed) { + assert lock.isHeldByCurrentThread(); this.ramBytesUsed += ramBytesUsed; } @@ -212,6 +218,7 @@ public class LRUQueryCache implements QueryCache, Accountable { * @lucene.experimental */ protected void onQueryEviction(Query query, long ramBytesUsed) { + assert lock.isHeldByCurrentThread(); this.ramBytesUsed -= ramBytesUsed; } @@ -223,6 +230,7 @@ public class LRUQueryCache implements QueryCache, Accountable { * @lucene.experimental */ protected void onDocIdSetCache(Object readerCoreKey, long ramBytesUsed) { + assert lock.isHeldByCurrentThread(); cacheSize += 1; cacheCount += 1; this.ramBytesUsed += ramBytesUsed; @@ -235,6 +243,7 @@ public class LRUQueryCache implements QueryCache, Accountable { * @lucene.experimental */ protected void onDocIdSetEviction(Object readerCoreKey, int numEntries, long sumRamBytesUsed) { + assert lock.isHeldByCurrentThread(); this.ramBytesUsed -= sumRamBytesUsed; cacheSize -= numEntries; } @@ -244,12 +253,14 @@ public class LRUQueryCache implements QueryCache, Accountable { * @lucene.experimental */ protected void onClear() { + assert lock.isHeldByCurrentThread(); ramBytesUsed = 0; cacheSize = 0; } /** Whether evictions are required. */ boolean requiresEviction() { + assert lock.isHeldByCurrentThread(); final int size = mostRecentlyUsedQueries.size(); if (size == 0) { return false; @@ -258,7 +269,8 @@ public class LRUQueryCache implements QueryCache, Accountable { } } - synchronized DocIdSet get(Query key, LeafReaderContext context) { + DocIdSet get(Query key, LeafReaderContext context) { + assert lock.isHeldByCurrentThread(); assert key instanceof BoostQuery == false; assert key instanceof ConstantScoreQuery == false; final Object readerKey = context.reader().getCoreCacheKey(); @@ -282,40 +294,45 @@ public class LRUQueryCache implements QueryCache, Accountable { return cached; } - synchronized void putIfAbsent(Query query, LeafReaderContext context, DocIdSet set) { - // under a lock to make sure that mostRecentlyUsedQueries and cache remain sync'ed - // we don't want to have user-provided queries as keys in our cache since queries are mutable + void putIfAbsent(Query query, LeafReaderContext context, DocIdSet set) { assert query instanceof BoostQuery == false; assert query instanceof ConstantScoreQuery == false; - Query singleton = uniqueQueries.putIfAbsent(query, query); - if (singleton == null) { - onQueryCache(singleton, LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY + ramBytesUsed(query)); - } else { - query = singleton; + // under a lock to make sure that mostRecentlyUsedQueries and cache remain sync'ed + lock.lock(); + try { + Query singleton = uniqueQueries.putIfAbsent(query, query); + if (singleton == null) { + onQueryCache(singleton, LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY + ramBytesUsed(query)); + } else { + query = singleton; + } + final Object key = context.reader().getCoreCacheKey(); + LeafCache leafCache = cache.get(key); + if (leafCache == null) { + leafCache = new LeafCache(key); + final LeafCache previous = cache.put(context.reader().getCoreCacheKey(), leafCache); + ramBytesUsed += HASHTABLE_RAM_BYTES_PER_ENTRY; + assert previous == null; + // we just created a new leaf cache, need to register a close listener + context.reader().addCoreClosedListener(new CoreClosedListener() { + @Override + public void onClose(Object ownerCoreCacheKey) { + clearCoreCacheKey(ownerCoreCacheKey); + } + }); + } + leafCache.putIfAbsent(query, set); + evictIfNecessary(); + } finally { + lock.unlock(); } - final Object key = context.reader().getCoreCacheKey(); - LeafCache leafCache = cache.get(key); - if (leafCache == null) { - leafCache = new LeafCache(key); - final LeafCache previous = cache.put(context.reader().getCoreCacheKey(), leafCache); - ramBytesUsed += HASHTABLE_RAM_BYTES_PER_ENTRY; - assert previous == null; - // we just created a new leaf cache, need to register a close listener - context.reader().addCoreClosedListener(new CoreClosedListener() { - @Override - public void onClose(Object ownerCoreCacheKey) { - clearCoreCacheKey(ownerCoreCacheKey); - } - }); - } - leafCache.putIfAbsent(query, set); - evictIfNecessary(); } - synchronized void evictIfNecessary() { + void evictIfNecessary() { + assert lock.isHeldByCurrentThread(); // under a lock to make sure that mostRecentlyUsedQueries and cache keep sync'ed if (requiresEviction()) { - + Iterator iterator = mostRecentlyUsedQueries.iterator(); do { final Query query = iterator.next(); @@ -337,31 +354,42 @@ public class LRUQueryCache implements QueryCache, Accountable { /** * Remove all cache entries for the given core cache key. */ - public synchronized void clearCoreCacheKey(Object coreKey) { - final LeafCache leafCache = cache.remove(coreKey); - if (leafCache != null) { - ramBytesUsed -= HASHTABLE_RAM_BYTES_PER_ENTRY; - final int numEntries = leafCache.cache.size(); - if (numEntries > 0) { - onDocIdSetEviction(coreKey, numEntries, leafCache.ramBytesUsed); - } else { - assert numEntries == 0; - assert leafCache.ramBytesUsed == 0; + public void clearCoreCacheKey(Object coreKey) { + lock.lock(); + try { + final LeafCache leafCache = cache.remove(coreKey); + if (leafCache != null) { + ramBytesUsed -= HASHTABLE_RAM_BYTES_PER_ENTRY; + final int numEntries = leafCache.cache.size(); + if (numEntries > 0) { + onDocIdSetEviction(coreKey, numEntries, leafCache.ramBytesUsed); + } else { + assert numEntries == 0; + assert leafCache.ramBytesUsed == 0; + } } + } finally { + lock.unlock(); } } /** * Remove all cache entries for the given query. */ - public synchronized void clearQuery(Query query) { - final Query singleton = uniqueQueries.remove(query); - if (singleton != null) { - onEviction(singleton); + public void clearQuery(Query query) { + lock.lock(); + try { + final Query singleton = uniqueQueries.remove(query); + if (singleton != null) { + onEviction(singleton); + } + } finally { + lock.unlock(); } } private void onEviction(Query singleton) { + assert lock.isHeldByCurrentThread(); onQueryEviction(singleton, LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY + ramBytesUsed(singleton)); for (LeafCache leafCache : cache.values()) { leafCache.remove(singleton); @@ -371,55 +399,70 @@ public class LRUQueryCache implements QueryCache, Accountable { /** * Clear the content of this cache. */ - public synchronized void clear() { - cache.clear(); - mostRecentlyUsedQueries.clear(); - onClear(); + public void clear() { + lock.lock(); + try { + cache.clear(); + mostRecentlyUsedQueries.clear(); + onClear(); + } finally { + lock.unlock(); + } } // pkg-private for testing - synchronized void assertConsistent() { - if (requiresEviction()) { - throw new AssertionError("requires evictions: size=" + mostRecentlyUsedQueries.size() - + ", maxSize=" + maxSize + ", ramBytesUsed=" + ramBytesUsed() + ", maxRamBytesUsed=" + maxRamBytesUsed); - } - for (LeafCache leafCache : cache.values()) { - Set keys = Collections.newSetFromMap(new IdentityHashMap<>()); - keys.addAll(leafCache.cache.keySet()); - keys.removeAll(mostRecentlyUsedQueries); - if (!keys.isEmpty()) { - throw new AssertionError("One leaf cache contains more keys than the top-level cache: " + keys); + void assertConsistent() { + lock.lock(); + try { + if (requiresEviction()) { + throw new AssertionError("requires evictions: size=" + mostRecentlyUsedQueries.size() + + ", maxSize=" + maxSize + ", ramBytesUsed=" + ramBytesUsed() + ", maxRamBytesUsed=" + maxRamBytesUsed); } - } - long recomputedRamBytesUsed = - HASHTABLE_RAM_BYTES_PER_ENTRY * cache.size() - + LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY * uniqueQueries.size(); - for (Query query : mostRecentlyUsedQueries) { - recomputedRamBytesUsed += ramBytesUsed(query); - } - for (LeafCache leafCache : cache.values()) { - recomputedRamBytesUsed += HASHTABLE_RAM_BYTES_PER_ENTRY * leafCache.cache.size(); - for (DocIdSet set : leafCache.cache.values()) { - recomputedRamBytesUsed += set.ramBytesUsed(); + for (LeafCache leafCache : cache.values()) { + Set keys = Collections.newSetFromMap(new IdentityHashMap<>()); + keys.addAll(leafCache.cache.keySet()); + keys.removeAll(mostRecentlyUsedQueries); + if (!keys.isEmpty()) { + throw new AssertionError("One leaf cache contains more keys than the top-level cache: " + keys); + } + } + long recomputedRamBytesUsed = + HASHTABLE_RAM_BYTES_PER_ENTRY * cache.size() + + LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY * uniqueQueries.size(); + for (Query query : mostRecentlyUsedQueries) { + recomputedRamBytesUsed += ramBytesUsed(query); + } + for (LeafCache leafCache : cache.values()) { + recomputedRamBytesUsed += HASHTABLE_RAM_BYTES_PER_ENTRY * leafCache.cache.size(); + for (DocIdSet set : leafCache.cache.values()) { + recomputedRamBytesUsed += set.ramBytesUsed(); + } + } + if (recomputedRamBytesUsed != ramBytesUsed) { + throw new AssertionError("ramBytesUsed mismatch : " + ramBytesUsed + " != " + recomputedRamBytesUsed); } - } - if (recomputedRamBytesUsed != ramBytesUsed) { - throw new AssertionError("ramBytesUsed mismatch : " + ramBytesUsed + " != " + recomputedRamBytesUsed); - } - long recomputedCacheSize = 0; - for (LeafCache leafCache : cache.values()) { - recomputedCacheSize += leafCache.cache.size(); - } - if (recomputedCacheSize != getCacheSize()) { - throw new AssertionError("cacheSize mismatch : " + getCacheSize() + " != " + recomputedCacheSize); + long recomputedCacheSize = 0; + for (LeafCache leafCache : cache.values()) { + recomputedCacheSize += leafCache.cache.size(); + } + if (recomputedCacheSize != getCacheSize()) { + throw new AssertionError("cacheSize mismatch : " + getCacheSize() + " != " + recomputedCacheSize); + } + } finally { + lock.unlock(); } } // pkg-private for testing // return the list of cached queries in LRU order - synchronized List cachedQueries() { - return new ArrayList<>(mostRecentlyUsedQueries); + List cachedQueries() { + lock.lock(); + try { + return new ArrayList<>(mostRecentlyUsedQueries); + } finally { + lock.unlock(); + } } @Override @@ -438,8 +481,11 @@ public class LRUQueryCache implements QueryCache, Accountable { @Override public Collection getChildResources() { - synchronized (this) { + lock.lock(); + try { return Accountables.namedAccountables("segment", cache); + } finally { + lock.unlock(); } } @@ -659,7 +705,18 @@ public class LRUQueryCache implements QueryCache, Accountable { return in.scorer(context); } - DocIdSet docIdSet = get(in.getQuery(), context); + // If the lock is already busy, prefer using the uncached version than waiting + if (lock.tryLock() == false) { + return in.scorer(context); + } + + DocIdSet docIdSet; + try { + docIdSet = get(in.getQuery(), context); + } finally { + lock.unlock(); + } + if (docIdSet == null) { if (policy.shouldCache(in.getQuery())) { docIdSet = cache(context); @@ -692,7 +749,18 @@ public class LRUQueryCache implements QueryCache, Accountable { return in.bulkScorer(context); } - DocIdSet docIdSet = get(in.getQuery(), context); + // If the lock is already busy, prefer using the uncached version than waiting + if (lock.tryLock() == false) { + return in.bulkScorer(context); + } + + DocIdSet docIdSet; + try { + docIdSet = get(in.getQuery(), context); + } finally { + lock.unlock(); + } + if (docIdSet == null) { if (policy.shouldCache(in.getQuery())) { docIdSet = cache(context);