diff --git a/lucene/core/src/java/org/apache/lucene/search/FilterCache.java b/lucene/core/src/java/org/apache/lucene/search/FilterCache.java index 23292a5cd77..8c4a97ea505 100644 --- a/lucene/core/src/java/org/apache/lucene/search/FilterCache.java +++ b/lucene/core/src/java/org/apache/lucene/search/FilterCache.java @@ -20,6 +20,7 @@ package org.apache.lucene.search; /** * A cache for filters. * + * @see LRUFilterCache * @lucene.experimental */ public interface FilterCache { diff --git a/lucene/core/src/java/org/apache/lucene/search/FilterCachingPolicy.java b/lucene/core/src/java/org/apache/lucene/search/FilterCachingPolicy.java index 107f4ec64a4..d9eb38b49a1 100644 --- a/lucene/core/src/java/org/apache/lucene/search/FilterCachingPolicy.java +++ b/lucene/core/src/java/org/apache/lucene/search/FilterCachingPolicy.java @@ -29,6 +29,8 @@ import org.apache.lucene.index.TieredMergePolicy; * * Implementations of this class must be thread-safe. * + * @see UsageTrackingFilterCachingPolicy + * @see LRUFilterCache * @lucene.experimental */ // TODO: add APIs for integration with IndexWriter.IndexReaderWarmer diff --git a/lucene/core/src/java/org/apache/lucene/search/LRUFilterCache.java b/lucene/core/src/java/org/apache/lucene/search/LRUFilterCache.java index aff78c9ac34..0e9c0ad0010 100644 --- a/lucene/core/src/java/org/apache/lucene/search/LRUFilterCache.java +++ b/lucene/core/src/java/org/apache/lucene/search/LRUFilterCache.java @@ -49,6 +49,38 @@ import org.apache.lucene.util.RoaringDocIdSet; * {@link FilterCachingPolicy caching policies} that only cache on "large" * segments, and it is advised to not share this cache across too many indices. * + * Typical usage looks like this: + *
+ *   final int maxNumberOfCachedFilters = 256;
+ *   final long maxRamBytesUsed = 50 * 1024L * 1024L; // 50MB
+ *   // these cache and policy instances can be shared across several filters and readers
+ *   // it is fine to eg. store them into static variables
+ *   final FilterCache filterCache = new LRUFilterCache(maxNumberOfCachedFilters, maxRamBytesUsed);
+ *   final FilterCachingPolicy defaultCachingPolicy = new UsageTrackingFilterCachingPolicy();
+ *   
+ *   // ...
+ *   
+ *   // Then at search time
+ *   Filter myFilter = ...;
+ *   Filter myCacheFilter = filterCache.doCache(myFilter, defaultCachingPolicy);
+ *   // myCacheFilter is now a wrapper around the original filter that will interact with the cache
+ *   IndexSearcher searcher = ...;
+ *   TopDocs topDocs = searcher.search(new ConstantScoreQuery(myCacheFilter), 10);
+ * 
+ * + * This cache exposes some global statistics ({@link #getHitCount() hit count}, + * {@link #getMissCount() miss count}, {@link #getCacheSize() number of cache + * entries}, {@link #getCacheCount() total number of DocIdSets that have ever + * been cached}, {@link #getEvictionCount() number of evicted entries}). In + * case you would like to have more fine-grained statistics, such as per-index + * or per-filter-class statistics, it is possible to override various callbacks: + * {@link #onHit}, {@link #onMiss}, + * {@link #onFilterCache}, {@link #onFilterEviction}, + * {@link #onDocIdSetCache}, {@link #onDocIdSetEviction} and {@link #onClear}. + * It is better to not perform heavy computations in these methods though since + * they are called synchronously and under a lock. + * + * @see FilterCachingPolicy * @lucene.experimental */ public class LRUFilterCache implements FilterCache, Accountable { @@ -96,6 +128,80 @@ public class LRUFilterCache implements FilterCache, Accountable { ramBytesUsed = 0; } + /** + * Expert: callback when there is a cache hit on a given filter. + * Implementing this method is typically useful in order to compute more + * fine-grained statistics about the filter cache. + * @see #onMiss + * @lucene.experimental + */ + protected void onHit(Object readerCoreKey, Filter filter) { + hitCount += 1; + } + + /** + * Expert: callback when there is a cache miss on a given filter. + * @see #onHit + * @lucene.experimental + */ + protected void onMiss(Object readerCoreKey, Filter filter) { + assert filter != null; + missCount += 1; + } + + /** + * Expert: callback when a filter is added to this cache. + * Implementing this method is typically useful in order to compute more + * fine-grained statistics about the filter cache. + * @see #onFilterEviction + * @lucene.experimental + */ + protected void onFilterCache(Filter filter, long ramBytesUsed) { + this.ramBytesUsed += ramBytesUsed; + } + + /** + * Expert: callback when a filter is evicted from this cache. + * @see #onFilterCache + * @lucene.experimental + */ + protected void onFilterEviction(Filter filter, long ramBytesUsed) { + this.ramBytesUsed -= ramBytesUsed; + } + + /** + * Expert: callback when a {@link DocIdSet} is added to this cache. + * Implementing this method is typically useful in order to compute more + * fine-grained statistics about the filter cache. + * @see #onDocIdSetEviction + * @lucene.experimental + */ + protected void onDocIdSetCache(Object readerCoreKey, long ramBytesUsed) { + cacheSize += 1; + cacheCount += 1; + this.ramBytesUsed += ramBytesUsed; + } + + /** + * Expert: callback when one or more {@link DocIdSet}s are removed from this + * cache. + * @see #onDocIdSetCache + * @lucene.experimental + */ + protected void onDocIdSetEviction(Object readerCoreKey, int numEntries, long sumRamBytesUsed) { + this.ramBytesUsed -= sumRamBytesUsed; + cacheSize -= numEntries; + } + + /** + * Expert: callback when the cache is completely cleared. + * @lucene.experimental + */ + protected void onClear() { + ramBytesUsed = 0; + cacheSize = 0; + } + /** Whether evictions are required. */ boolean requiresEviction() { final int size = mostRecentlyUsedFilters.size(); @@ -107,22 +213,23 @@ public class LRUFilterCache implements FilterCache, Accountable { } synchronized DocIdSet get(Filter filter, LeafReaderContext context) { - final LeafCache leafCache = cache.get(context.reader().getCoreCacheKey()); + final Object readerKey = context.reader().getCoreCacheKey(); + final LeafCache leafCache = cache.get(readerKey); if (leafCache == null) { - missCount += 1; + onMiss(readerKey, filter); return null; } // this get call moves the filter to the most-recently-used position final Filter singleton = uniqueFilters.get(filter); if (singleton == null) { - missCount += 1; + onMiss(readerKey, filter); return null; } final DocIdSet cached = leafCache.get(singleton); if (cached == null) { - missCount += 1; + onMiss(readerKey, singleton); } else { - hitCount += 1; + onHit(readerKey, singleton); } return cached; } @@ -132,13 +239,14 @@ public class LRUFilterCache implements FilterCache, Accountable { assert set.isCacheable(); Filter singleton = uniqueFilters.putIfAbsent(filter, filter); if (singleton == null) { - ramBytesUsed += LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY + ramBytesUsed(filter); + onFilterCache(singleton, LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY + ramBytesUsed(filter)); } else { filter = singleton; } - LeafCache leafCache = cache.get(context.reader().getCoreCacheKey()); + final Object key = context.reader().getCoreCacheKey(); + LeafCache leafCache = cache.get(key); if (leafCache == null) { - leafCache = new LeafCache(); + leafCache = new LeafCache(key); final LeafCache previous = cache.put(context.reader().getCoreCacheKey(), leafCache); ramBytesUsed += HASHTABLE_RAM_BYTES_PER_ENTRY; assert previous == null; @@ -172,8 +280,8 @@ public class LRUFilterCache implements FilterCache, Accountable { public synchronized void clearCoreCacheKey(Object coreKey) { final LeafCache leafCache = cache.remove(coreKey); if (leafCache != null) { - ramBytesUsed -= leafCache.ramBytesUsed + HASHTABLE_RAM_BYTES_PER_ENTRY; - cacheSize -= leafCache.cache.size(); + ramBytesUsed -= HASHTABLE_RAM_BYTES_PER_ENTRY; + onDocIdSetEviction(coreKey, leafCache.cache.size(), leafCache.ramBytesUsed); } } @@ -188,7 +296,7 @@ public class LRUFilterCache implements FilterCache, Accountable { } private void onEviction(Filter singleton) { - ramBytesUsed -= LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY + ramBytesUsed(singleton); + onFilterEviction(singleton, LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY + ramBytesUsed(singleton)); for (LeafCache leafCache : cache.values()) { leafCache.remove(singleton); } @@ -200,8 +308,7 @@ public class LRUFilterCache implements FilterCache, Accountable { public synchronized void clear() { cache.clear(); mostRecentlyUsedFilters.clear(); - ramBytesUsed = 0; - cacheSize = 0; + onClear(); } // pkg-private for testing @@ -388,17 +495,24 @@ public class LRUFilterCache implements FilterCache, Accountable { // this class is not thread-safe, everything but ramBytesUsed needs to be called under a lock private class LeafCache implements Accountable { + private final Object key; private final Map cache; private volatile long ramBytesUsed; - LeafCache() { + LeafCache(Object key) { + this.key = key; cache = new IdentityHashMap<>(); ramBytesUsed = 0; } - private void incrementRamBytesUsed(long inc) { - ramBytesUsed += inc; - LRUFilterCache.this.ramBytesUsed += inc; + private void onDocIdSetCache(long ramBytesUsed) { + this.ramBytesUsed += ramBytesUsed; + LRUFilterCache.this.onDocIdSetCache(key, ramBytesUsed); + } + + private void onDocIdSetEviction(long ramBytesUsed) { + this.ramBytesUsed -= ramBytesUsed; + LRUFilterCache.this.onDocIdSetEviction(key, 1, ramBytesUsed); } DocIdSet get(Filter filter) { @@ -408,17 +522,14 @@ public class LRUFilterCache implements FilterCache, Accountable { void putIfAbsent(Filter filter, DocIdSet set) { if (cache.putIfAbsent(filter, set) == null) { // the set was actually put - cacheCount += 1; - cacheSize += 1; - incrementRamBytesUsed(HASHTABLE_RAM_BYTES_PER_ENTRY + set.ramBytesUsed()); + onDocIdSetCache(HASHTABLE_RAM_BYTES_PER_ENTRY + set.ramBytesUsed()); } } void remove(Filter filter) { DocIdSet removed = cache.remove(filter); if (removed != null) { - cacheSize -= 1; - incrementRamBytesUsed(-(HASHTABLE_RAM_BYTES_PER_ENTRY + removed.ramBytesUsed())); + onDocIdSetEviction(HASHTABLE_RAM_BYTES_PER_ENTRY + removed.ramBytesUsed()); } } diff --git a/lucene/core/src/test/org/apache/lucene/search/TestLRUFilterCache.java b/lucene/core/src/test/org/apache/lucene/search/TestLRUFilterCache.java index 81ba9d9c008..07cbc40a4f3 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestLRUFilterCache.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestLRUFilterCache.java @@ -27,6 +27,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import org.apache.lucene.document.Document; @@ -544,4 +545,169 @@ public class TestLRUFilterCache extends LuceneTestCase { dir.close(); } + public void testFineGrainedStats() throws IOException { + Directory dir1 = newDirectory(); + final RandomIndexWriter w1 = new RandomIndexWriter(random(), dir1); + Directory dir2 = newDirectory(); + final RandomIndexWriter w2 = new RandomIndexWriter(random(), dir2); + + final List colors = Arrays.asList("blue", "red", "green", "yellow"); + + Document doc = new Document(); + StringField f = new StringField("color", "", Store.NO); + doc.add(f); + for (RandomIndexWriter w : Arrays.asList(w1, w2)) { + for (int i = 0; i < 10; ++i) { + f.setStringValue(RandomPicks.randomFrom(random(), colors)); + w.addDocument(doc); + if (random().nextBoolean()) { + w.getReader().close(); + } + } + } + + final DirectoryReader reader1 = w1.getReader(); + final int segmentCount1 = reader1.leaves().size(); + final IndexSearcher searcher1 = new IndexSearcher(reader1); + + final DirectoryReader reader2 = w2.getReader(); + final int segmentCount2 = reader2.leaves().size(); + final IndexSearcher searcher2 = new IndexSearcher(reader2); + + final Map indexId = new HashMap<>(); + for (LeafReaderContext ctx : reader1.leaves()) { + indexId.put(ctx.reader().getCoreCacheKey(), 1); + } + for (LeafReaderContext ctx : reader2.leaves()) { + indexId.put(ctx.reader().getCoreCacheKey(), 2); + } + + final AtomicLong hitCount1 = new AtomicLong(); + final AtomicLong hitCount2 = new AtomicLong(); + final AtomicLong missCount1 = new AtomicLong(); + final AtomicLong missCount2 = new AtomicLong(); + + final AtomicLong ramBytesUsage = new AtomicLong(); + final AtomicLong cacheSize = new AtomicLong(); + + final LRUFilterCache filterCache = new LRUFilterCache(2, 10000000) { + @Override + protected void onHit(Object readerCoreKey, Filter filter) { + super.onHit(readerCoreKey, filter); + switch(indexId.get(readerCoreKey).intValue()) { + case 1: + hitCount1.incrementAndGet(); + break; + case 2: + hitCount2.incrementAndGet(); + break; + default: + throw new AssertionError(); + } + } + + @Override + protected void onMiss(Object readerCoreKey, Filter filter) { + super.onMiss(readerCoreKey, filter); + switch(indexId.get(readerCoreKey).intValue()) { + case 1: + missCount1.incrementAndGet(); + break; + case 2: + missCount2.incrementAndGet(); + break; + default: + throw new AssertionError(); + } + } + + @Override + protected void onFilterCache(Filter filter, long ramBytesUsed) { + super.onFilterCache(filter, ramBytesUsed); + ramBytesUsage.addAndGet(ramBytesUsed); + } + + @Override + protected void onFilterEviction(Filter filter, long ramBytesUsed) { + super.onFilterEviction(filter, ramBytesUsed); + ramBytesUsage.addAndGet(-ramBytesUsed); + } + + @Override + protected void onDocIdSetCache(Object readerCoreKey, long ramBytesUsed) { + super.onDocIdSetCache(readerCoreKey, ramBytesUsed); + ramBytesUsage.addAndGet(ramBytesUsed); + cacheSize.incrementAndGet(); + } + + @Override + protected void onDocIdSetEviction(Object readerCoreKey, int numEntries, long sumRamBytesUsed) { + super.onDocIdSetEviction(readerCoreKey, numEntries, sumRamBytesUsed); + ramBytesUsage.addAndGet(-sumRamBytesUsed); + cacheSize.addAndGet(-numEntries); + } + + @Override + protected void onClear() { + super.onClear(); + ramBytesUsage.set(0); + cacheSize.set(0); + } + }; + + final Filter filter = new QueryWrapperFilter(new TermQuery(new Term("color", "red"))); + final Filter filter2 = new QueryWrapperFilter(new TermQuery(new Term("color", "blue"))); + final Filter filter3 = new QueryWrapperFilter(new TermQuery(new Term("color", "green"))); + + // search on searcher1 + Filter cached = filterCache.doCache(filter, FilterCachingPolicy.ALWAYS_CACHE); + for (int i = 0; i < 10; ++i) { + searcher1.search(new ConstantScoreQuery(cached), 1); + } + assertEquals(9 * segmentCount1, hitCount1.longValue()); + assertEquals(0, hitCount2.longValue()); + assertEquals(segmentCount1, missCount1.longValue()); + assertEquals(0, missCount2.longValue()); + + // then on searcher2 + cached = filterCache.doCache(filter2, FilterCachingPolicy.ALWAYS_CACHE); + for (int i = 0; i < 20; ++i) { + searcher2.search(new ConstantScoreQuery(cached), 1); + } + assertEquals(9 * segmentCount1, hitCount1.longValue()); + assertEquals(19 * segmentCount2, hitCount2.longValue()); + assertEquals(segmentCount1, missCount1.longValue()); + assertEquals(segmentCount2, missCount2.longValue()); + + // now on searcher1 again to trigger evictions + cached = filterCache.doCache(filter3, FilterCachingPolicy.ALWAYS_CACHE); + for (int i = 0; i < 30; ++i) { + searcher1.search(new ConstantScoreQuery(cached), 1); + } + assertEquals(segmentCount1, filterCache.getEvictionCount()); + assertEquals(38 * segmentCount1, hitCount1.longValue()); + assertEquals(19 * segmentCount2, hitCount2.longValue()); + assertEquals(2 * segmentCount1, missCount1.longValue()); + assertEquals(segmentCount2, missCount2.longValue()); + + // check that the recomputed stats are the same as those reported by the cache + assertEquals(filterCache.ramBytesUsed(), (segmentCount1 + segmentCount2) * LRUFilterCache.HASHTABLE_RAM_BYTES_PER_ENTRY + ramBytesUsage.longValue()); + assertEquals(filterCache.getCacheSize(), cacheSize.longValue()); + + reader1.close(); + reader2.close(); + w1.close(); + w2.close(); + + assertEquals(filterCache.ramBytesUsed(), ramBytesUsage.longValue()); + assertEquals(0, cacheSize.longValue()); + + filterCache.clear(); + assertEquals(0, ramBytesUsage.longValue()); + assertEquals(0, cacheSize.longValue()); + + dir1.close(); + dir2.close(); + } + }