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 2014/12/11 10:50:55 UTC
svn commit: r1644582 - in /lucene/dev/trunk/lucene/core/src:
java/org/apache/lucene/search/LRUFilterCache.java
test/org/apache/lucene/search/TestLRUFilterCache.java
Author: jpountz
Date: Thu Dec 11 09:50:54 2014
New Revision: 1644582
URL: http://svn.apache.org/r1644582
Log:
LUCENE-6096: Improve and test LRUFilterCache.ramBytesUsed.
Modified:
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/LRUFilterCache.java
lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestLRUFilterCache.java
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/LRUFilterCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/LRUFilterCache.java?rev=1644582&r1=1644581&r2=1644582&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/LRUFilterCache.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/LRUFilterCache.java Thu Dec 11 09:50:54 2014
@@ -18,12 +18,12 @@ package org.apache.lucene.search;
*/
import java.io.IOException;
+import java.util.ArrayList;
import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
import java.util.IdentityHashMap;
import java.util.Iterator;
import java.util.LinkedHashMap;
+import java.util.List;
import java.util.Map;
import java.util.Set;
@@ -33,6 +33,7 @@ import org.apache.lucene.index.LeafReade
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.RoaringDocIdSet;
/**
@@ -51,8 +52,22 @@ import org.apache.lucene.util.RoaringDoc
*/
public class LRUFilterCache implements FilterCache, Accountable {
+ // memory usage of a simple query-wrapper filter around a term query
+ static final long FILTER_DEFAULT_RAM_BYTES_USED = 216;
+
+ static final long HASHTABLE_RAM_BYTES_PER_ENTRY =
+ 2 * RamUsageEstimator.NUM_BYTES_OBJECT_REF // key + value
+ * 2; // hash tables need to be oversized to avoid collisions, assume 2x capacity
+
+ static final long LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY =
+ HASHTABLE_RAM_BYTES_PER_ENTRY
+ + 2 * RamUsageEstimator.NUM_BYTES_OBJECT_REF; // previous & next references
+
private final int maxSize;
private final long maxRamBytesUsed;
+ // maps filters that are contained in the cache to a singleton so that this
+ // cache does not store several copies of the same filter
+ private final Map<Filter, Filter> uniqueFilters;
// The contract between this set and the per-leaf caches is that per-leaf caches
// are only allowed to store sub-sets of the filters that are contained in
// mostRecentlyUsedFilters. This is why write operations are performed under a lock
@@ -67,14 +82,20 @@ public class LRUFilterCache implements F
public LRUFilterCache(int maxSize, long maxRamBytesUsed) {
this.maxSize = maxSize;
this.maxRamBytesUsed = maxRamBytesUsed;
- mostRecentlyUsedFilters = Collections.newSetFromMap(new LinkedHashMap<Filter, Boolean>(16, 0.75f, true));
+ uniqueFilters = new LinkedHashMap<Filter, Filter>(16, 0.75f, true);
+ mostRecentlyUsedFilters = uniqueFilters.keySet();
cache = new IdentityHashMap<>();
ramBytesUsed = 0;
}
/** Whether evictions are required. */
boolean requiresEviction() {
- return mostRecentlyUsedFilters.size() > maxSize || ramBytesUsed() > maxRamBytesUsed;
+ final int size = mostRecentlyUsedFilters.size();
+ if (size == 0) {
+ return false;
+ } else {
+ return size > maxSize || ramBytesUsed() > maxRamBytesUsed;
+ }
}
synchronized DocIdSet get(Filter filter, LeafReaderContext context) {
@@ -82,27 +103,28 @@ public class LRUFilterCache implements F
if (leafCache == null) {
return null;
}
- final DocIdSet set = leafCache.get(filter);
- if (set != null) {
- // this filter becomes the most-recently used filter
- final boolean added = mostRecentlyUsedFilters.add(filter);
- // added is necessarily false since the leaf caches contain a subset of mostRecentlyUsedFilters
- assert added == false;
+ // this get call moves the filter to the most-recently-used position
+ final Filter singleton = uniqueFilters.get(filter);
+ if (singleton == null) {
+ return null;
}
- return set;
+ return leafCache.get(singleton);
}
synchronized void putIfAbsent(Filter filter, LeafReaderContext context, DocIdSet set) {
// under a lock to make sure that mostRecentlyUsedFilters and cache remain sync'ed
assert set.isCacheable();
- final boolean added = mostRecentlyUsedFilters.add(filter);
- if (added) {
- ramBytesUsed += ramBytesUsed(filter);
+ Filter singleton = uniqueFilters.putIfAbsent(filter, filter);
+ if (singleton == null) {
+ ramBytesUsed += LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY + ramBytesUsed(filter);
+ } else {
+ filter = singleton;
}
LeafCache leafCache = cache.get(context.reader().getCoreCacheKey());
if (leafCache == null) {
leafCache = new LeafCache();
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() {
@@ -118,13 +140,12 @@ public class LRUFilterCache implements F
synchronized void evictIfNecessary() {
// under a lock to make sure that mostRecentlyUsedFilters and cache keep sync'ed
- if (requiresEviction() && mostRecentlyUsedFilters.isEmpty() == false) {
+ if (requiresEviction()) {
Iterator<Filter> iterator = mostRecentlyUsedFilters.iterator();
do {
final Filter filter = iterator.next();
iterator.remove();
- ramBytesUsed -= ramBytesUsed(filter);
- clearFilter(filter);
+ onEviction(filter);
} while (iterator.hasNext() && requiresEviction());
}
}
@@ -135,7 +156,7 @@ public class LRUFilterCache implements F
public synchronized void clearCoreCacheKey(Object coreKey) {
final LeafCache leafCache = cache.remove(coreKey);
if (leafCache != null) {
- ramBytesUsed -= leafCache.ramBytesUsed;
+ ramBytesUsed -= leafCache.ramBytesUsed + HASHTABLE_RAM_BYTES_PER_ENTRY;
}
}
@@ -143,8 +164,16 @@ public class LRUFilterCache implements F
* Remove all cache entries for the given filter.
*/
public synchronized void clearFilter(Filter filter) {
+ final Filter singleton = uniqueFilters.remove(filter);
+ if (singleton != null) {
+ onEviction(singleton);
+ }
+ }
+
+ private void onEviction(Filter singleton) {
+ ramBytesUsed -= LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY + ramBytesUsed(singleton);
for (LeafCache leafCache : cache.values()) {
- leafCache.remove(filter);
+ leafCache.remove(singleton);
}
}
@@ -164,18 +193,24 @@ public class LRUFilterCache implements F
+ ", maxSize=" + maxSize + ", ramBytesUsed=" + ramBytesUsed() + ", maxRamBytesUsed=" + maxRamBytesUsed);
}
for (LeafCache leafCache : cache.values()) {
- Set<Filter> keys = new HashSet<Filter>(leafCache.cache.keySet());
+ Set<Filter> keys = Collections.newSetFromMap(new IdentityHashMap<>());
+ keys.addAll(leafCache.cache.keySet());
keys.removeAll(mostRecentlyUsedFilters);
if (!keys.isEmpty()) {
throw new AssertionError("One leaf cache contains more keys than the top-level cache: " + keys);
}
}
- long recomputedRamBytesUsed = 0;
+ long recomputedRamBytesUsed =
+ HASHTABLE_RAM_BYTES_PER_ENTRY * cache.size()
+ + LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY * uniqueFilters.size();
for (Filter filter : mostRecentlyUsedFilters) {
recomputedRamBytesUsed += ramBytesUsed(filter);
}
for (LeafCache leafCache : cache.values()) {
- recomputedRamBytesUsed += leafCache.ramBytesUsed();
+ 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);
@@ -183,8 +218,9 @@ public class LRUFilterCache implements F
}
// pkg-private for testing
- synchronized Set<Filter> cachedFilters() {
- return new HashSet<>(mostRecentlyUsedFilters);
+ // return the list of cached filters in LRU order
+ synchronized List<Filter> cachedFilters() {
+ return new ArrayList<>(mostRecentlyUsedFilters);
}
@Override
@@ -240,7 +276,7 @@ public class LRUFilterCache implements F
if (filter instanceof Accountable) {
return ((Accountable) filter).ramBytesUsed();
}
- return 1024;
+ return FILTER_DEFAULT_RAM_BYTES_USED;
}
/**
@@ -257,7 +293,7 @@ public class LRUFilterCache implements F
private volatile long ramBytesUsed;
LeafCache() {
- cache = new HashMap<>();
+ cache = new IdentityHashMap<>();
ramBytesUsed = 0;
}
@@ -273,14 +309,14 @@ public class LRUFilterCache implements F
void putIfAbsent(Filter filter, DocIdSet set) {
if (cache.putIfAbsent(filter, set) == null) {
// the set was actually put
- incrementRamBytesUsed(set.ramBytesUsed());
+ incrementRamBytesUsed(HASHTABLE_RAM_BYTES_PER_ENTRY + set.ramBytesUsed());
}
}
void remove(Filter filter) {
DocIdSet removed = cache.remove(filter);
if (removed != null) {
- incrementRamBytesUsed(-removed.ramBytesUsed());
+ incrementRamBytesUsed(-(HASHTABLE_RAM_BYTES_PER_ENTRY + removed.ramBytesUsed()));
}
}
Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestLRUFilterCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestLRUFilterCache.java?rev=1644582&r1=1644581&r2=1644582&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestLRUFilterCache.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestLRUFilterCache.java Thu Dec 11 09:50:54 2014
@@ -18,9 +18,13 @@ package org.apache.lucene.search;
*/
import java.io.IOException;
+import java.lang.reflect.Field;
import java.util.Arrays;
+import java.util.Collection;
import java.util.Collections;
-import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
@@ -32,7 +36,9 @@ import org.apache.lucene.index.LeafReade
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.Term;
import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.Bits;
import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.RamUsageTester;
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
@@ -50,6 +56,10 @@ public class TestLRUFilterCache extends
};
+ public void testFilterRamBytesUsed() {
+ assertEquals(LRUFilterCache.FILTER_DEFAULT_RAM_BYTES_USED, RamUsageTester.sizeOf(new QueryWrapperFilter(new TermQuery(new Term("some_field", "some_term")))));
+ }
+
public void testConcurrency() throws Throwable {
final LRUFilterCache filterCache = new LRUFilterCache(1 + random().nextInt(20), 1 + random().nextInt(10000));
Directory dir = newDirectory();
@@ -151,32 +161,230 @@ public class TestLRUFilterCache extends
final Filter red = new QueryWrapperFilter(new TermQuery(new Term("color", "red")));
final Filter green = new QueryWrapperFilter(new TermQuery(new Term("color", "green")));
- assertEquals(Collections.emptySet(), filterCache.cachedFilters());
+ assertEquals(Collections.emptyList(), filterCache.cachedFilters());
// the filter is not cached on any segment: no changes
searcher.search(new ConstantScoreQuery(filterCache.doCache(green, NEVER_CACHE)), 1);
- assertEquals(Collections.emptySet(), filterCache.cachedFilters());
+ assertEquals(Collections.emptyList(), filterCache.cachedFilters());
searcher.search(new ConstantScoreQuery(filterCache.doCache(red, FilterCachingPolicy.ALWAYS_CACHE)), 1);
- assertEquals(Collections.singleton(red), filterCache.cachedFilters());
+ assertEquals(Collections.singletonList(red), filterCache.cachedFilters());
searcher.search(new ConstantScoreQuery(filterCache.doCache(green, FilterCachingPolicy.ALWAYS_CACHE)), 1);
- assertEquals(new HashSet<>(Arrays.asList(red, green)), filterCache.cachedFilters());
+ assertEquals(Arrays.asList(red, green), filterCache.cachedFilters());
searcher.search(new ConstantScoreQuery(filterCache.doCache(red, FilterCachingPolicy.ALWAYS_CACHE)), 1);
- assertEquals(new HashSet<>(Arrays.asList(red, green)), filterCache.cachedFilters());
+ assertEquals(Arrays.asList(green, red), filterCache.cachedFilters());
searcher.search(new ConstantScoreQuery(filterCache.doCache(blue, FilterCachingPolicy.ALWAYS_CACHE)), 1);
- assertEquals(new HashSet<>(Arrays.asList(red, blue)), filterCache.cachedFilters());
+ assertEquals(Arrays.asList(red, blue), filterCache.cachedFilters());
searcher.search(new ConstantScoreQuery(filterCache.doCache(blue, FilterCachingPolicy.ALWAYS_CACHE)), 1);
- assertEquals(new HashSet<>(Arrays.asList(red, blue)), filterCache.cachedFilters());
+ assertEquals(Arrays.asList(red, blue), filterCache.cachedFilters());
searcher.search(new ConstantScoreQuery(filterCache.doCache(green, FilterCachingPolicy.ALWAYS_CACHE)), 1);
- assertEquals(new HashSet<>(Arrays.asList(green, blue)), filterCache.cachedFilters());
+ assertEquals(Arrays.asList(blue, green), filterCache.cachedFilters());
searcher.search(new ConstantScoreQuery(filterCache.doCache(red, NEVER_CACHE)), 1);
- assertEquals(new HashSet<>(Arrays.asList(green, blue)), filterCache.cachedFilters());
+ assertEquals(Arrays.asList(blue, green), filterCache.cachedFilters());
+
+ reader.close();
+ w.close();
+ dir.close();
+ }
+
+ public void testCache() throws IOException {
+ Directory dir = newDirectory();
+ final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+
+ Document doc = new Document();
+ StringField f = new StringField("color", "", Store.NO);
+ doc.add(f);
+ final int numDocs = atLeast(10);
+ for (int i = 0; i < numDocs; ++i) {
+ f.setStringValue(RandomPicks.randomFrom(random(), Arrays.asList("blue", "red", "green")));
+ w.addDocument(doc);
+ }
+ final DirectoryReader reader = w.getReader();
+ final LeafReaderContext leaf1 = reader.leaves().get(0);
+
+ Filter filter1 = new QueryWrapperFilter(new TermQuery(new Term("color", "blue")));
+ // different instance yet equal
+ Filter filter2 = new QueryWrapperFilter(new TermQuery(new Term("color", "blue")));
+
+ final LRUFilterCache filterCache = new LRUFilterCache(Integer.MAX_VALUE, Long.MAX_VALUE);
+ final Filter cachedFilter1 = filterCache.doCache(filter1, FilterCachingPolicy.ALWAYS_CACHE);
+ DocIdSet cached1 = cachedFilter1.getDocIdSet(leaf1, null);
+
+ final Filter cachedFilter2 = filterCache.doCache(filter2, NEVER_CACHE);
+ DocIdSet cached2 = cachedFilter2.getDocIdSet(leaf1, null);
+ assertSame(cached1, cached2);
+
+ filterCache.assertConsistent();
+
+ reader.close();
+ w.close();
+ dir.close();
+ }
+
+ public void testClearFilter() throws IOException {
+ Directory dir = newDirectory();
+ final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+
+ Document doc = new Document();
+ StringField f = new StringField("color", "", Store.NO);
+ doc.add(f);
+ final int numDocs = atLeast(10);
+ for (int i = 0; i < numDocs; ++i) {
+ f.setStringValue(random().nextBoolean() ? "red" : "blue");
+ w.addDocument(doc);
+ }
+ final DirectoryReader reader = w.getReader();
+ final LeafReaderContext leaf1 = reader.leaves().get(0);
+
+ final Filter filter1 = new QueryWrapperFilter(new TermQuery(new Term("color", "blue")));
+ // different instance yet equal
+ final Filter filter2 = new QueryWrapperFilter(new TermQuery(new Term("color", "blue")));
+
+ final LRUFilterCache filterCache = new LRUFilterCache(Integer.MAX_VALUE, Long.MAX_VALUE);
+
+ final Filter cachedFilter1 = filterCache.doCache(filter1, FilterCachingPolicy.ALWAYS_CACHE);
+ cachedFilter1.getDocIdSet(leaf1, null);
+
+ filterCache.clearFilter(filter2);
+
+ assertTrue(filterCache.cachedFilters().isEmpty());
+ filterCache.assertConsistent();
+
+ reader.close();
+ w.close();
+ dir.close();
+ }
+
+ // This test makes sure that by making the same assumptions as LRUFilterCache, RAMUsageTester
+ // computes the same memory usage.
+ public void testRamBytesUsedAgreesWithRamUsageTester() throws IOException {
+ final LRUFilterCache filterCache = new LRUFilterCache(1 + random().nextInt(5), 1 + random().nextInt(10000));
+ // an accumulator that only sums up memory usage of referenced filters and doc id sets
+ final RamUsageTester.Accumulator acc = new RamUsageTester.Accumulator() {
+ @Override
+ public long accumulateObject(Object o, long shallowSize, Map<Field,Object> fieldValues, Collection<Object> queue) {
+ if (o instanceof DocIdSet) {
+ return ((DocIdSet) o).ramBytesUsed();
+ }
+ if (o instanceof Filter) {
+ return filterCache.ramBytesUsed((Filter) o);
+ }
+ if (o.getClass().getSimpleName().equals("SegmentCoreReaders")) {
+ // do not take core cache keys into account
+ return 0;
+ }
+ if (o instanceof Map) {
+ Map<?,?> map = (Map<?,?>) o;
+ queue.addAll(map.keySet());
+ queue.addAll(map.values());
+ final long sizePerEntry = o instanceof LinkedHashMap
+ ? LRUFilterCache.LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY
+ : LRUFilterCache.HASHTABLE_RAM_BYTES_PER_ENTRY;
+ return sizePerEntry * map.size();
+ }
+ // follow links to other objects, but ignore their memory usage
+ super.accumulateObject(o, shallowSize, fieldValues, queue);
+ return 0;
+ }
+ @Override
+ public long accumulateArray(Object array, long shallowSize, List<Object> values, Collection<Object> queue) {
+ // follow links to other objects, but ignore their memory usage
+ super.accumulateArray(array, shallowSize, values, queue);
+ return 0;
+ }
+ };
+
+ Directory dir = newDirectory();
+ final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+
+ final List<String> colors = Arrays.asList("blue", "red", "green", "yellow");
+
+ Document doc = new Document();
+ StringField f = new StringField("color", "", Store.NO);
+ doc.add(f);
+ final int iters = atLeast(5);
+ for (int iter = 0; iter < iters; ++iter) {
+ final int numDocs = atLeast(10);
+ for (int i = 0; i < numDocs; ++i) {
+ f.setStringValue(RandomPicks.randomFrom(random(), colors));
+ w.addDocument(doc);
+ }
+ try (final DirectoryReader reader = w.getReader()) {
+ final IndexSearcher searcher = new IndexSearcher(reader);
+ for (int i = 0; i < 3; ++i) {
+ final Filter filter = new QueryWrapperFilter(new TermQuery(new Term("color", RandomPicks.randomFrom(random(), colors))));
+ searcher.search(new ConstantScoreQuery(filterCache.doCache(filter, MAYBE_CACHE_POLICY)), 1);
+ }
+ }
+ filterCache.assertConsistent();
+ assertEquals(RamUsageTester.sizeOf(filterCache, acc), filterCache.ramBytesUsed());
+ }
+
+ w.close();
+ dir.close();
+ }
+
+ /** A filter that produces empty sets. */
+ private static class DummyFilter extends Filter {
+
+ @Override
+ public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
+ return null;
+ }
+
+ }
+
+ // Test what happens when the cache contains only filters and doc id sets
+ // that require very little memory. In that case most of the memory is taken
+ // 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 LRUFilterCache filterCache = new LRUFilterCache(1000000, 10000000);
+
+ final RamUsageTester.Accumulator acc = new RamUsageTester.Accumulator() {
+ @Override
+ public long accumulateObject(Object o, long shallowSize, Map<Field,Object> fieldValues, Collection<Object> queue) {
+ if (o instanceof DocIdSet) {
+ return ((DocIdSet) o).ramBytesUsed();
+ }
+ if (o instanceof Filter) {
+ return filterCache.ramBytesUsed((Filter) o);
+ }
+ if (o.getClass().getSimpleName().equals("SegmentCoreReaders")) {
+ // do not follow references to core cache keys
+ return 0;
+ }
+ return super.accumulateObject(o, shallowSize, fieldValues, queue);
+ }
+ };
+
+ Directory dir = newDirectory();
+ final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+ Document doc = new Document();
+ final int numDocs = atLeast(100);
+ for (int i = 0; i < numDocs; ++i) {
+ w.addDocument(doc);
+ }
+ final DirectoryReader reader = w.getReader();
+ final IndexSearcher searcher = new IndexSearcher(reader);
+
+ final int numFilters = atLeast(1000);
+ for (int i = 0; i < numFilters; ++i) {
+ final Filter filter = new DummyFilter();
+ final Filter cached = filterCache.doCache(filter, FilterCachingPolicy.ALWAYS_CACHE);
+ searcher.search(new ConstantScoreQuery(cached), 1);
+ }
+
+ final long actualRamBytesUsed = RamUsageTester.sizeOf(filterCache, acc);
+ final long expectedRamBytesUsed = filterCache.ramBytesUsed();
+ // error < 30%
+ assertEquals(actualRamBytesUsed, expectedRamBytesUsed, 30 * actualRamBytesUsed / 100);
reader.close();
w.close();