LUCENE-6077: Add a filter cache.

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1642183 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Adrien Grand 2014-11-27 16:23:44 +00:00
parent c539640102
commit 86a6c37b47
14 changed files with 1005 additions and 27 deletions

View File

@ -115,6 +115,8 @@ New Features
suggesters from multi-valued fields. (Varun Thacker via Mike
McCandless)
* LUCENE-6077: Added a filter cache. (Adrien Grand, Robert Muir)
API Changes
* LUCENE-5900: Deprecated more constructors taking Version in *InfixSuggester and

View File

@ -22,7 +22,6 @@ import static org.apache.lucene.search.DocIdSet.EMPTY;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.WeakHashMap;
@ -41,13 +40,23 @@ import org.apache.lucene.util.RoaringDocIdSet;
*/
public class CachingWrapperFilter extends Filter implements Accountable {
private final Filter filter;
private final FilterCachingPolicy policy;
private final Map<Object,DocIdSet> cache = Collections.synchronizedMap(new WeakHashMap<Object,DocIdSet>());
/** Wraps another filter's result and caches it.
/** Wraps another filter's result and caches it according to the provided policy.
* @param filter Filter to cache results of
* @param policy policy defining which filters should be cached on which segments
*/
public CachingWrapperFilter(Filter filter) {
public CachingWrapperFilter(Filter filter, FilterCachingPolicy policy) {
this.filter = filter;
this.policy = policy;
}
/** Same as {@link CachingWrapperFilter#CachingWrapperFilter(Filter, FilterCachingPolicy)}
* but enforces the use of the
* {@link FilterCachingPolicy.CacheOnLargeSegments#DEFAULT} policy. */
public CachingWrapperFilter(Filter filter) {
this(filter, FilterCachingPolicy.CacheOnLargeSegments.DEFAULT);
}
/**
@ -99,8 +108,10 @@ public class CachingWrapperFilter extends Filter implements Accountable {
if (docIdSet != null) {
hitCount++;
} else {
docIdSet = filter.getDocIdSet(context, null);
if (policy.shouldCache(filter, context, docIdSet)) {
missCount++;
docIdSet = docIdSetToCache(filter.getDocIdSet(context, null), reader);
docIdSet = docIdSetToCache(docIdSet, reader);
if (docIdSet == null) {
// We use EMPTY as a sentinel for the empty set, which is cacheable
docIdSet = EMPTY;
@ -108,6 +119,7 @@ public class CachingWrapperFilter extends Filter implements Accountable {
assert docIdSet.isCacheable();
cache.put(key, docIdSet);
}
}
return docIdSet == EMPTY ? null : BitsFilteredDocIdSet.wrap(docIdSet, acceptDocs);
}

View File

@ -0,0 +1,33 @@
package org.apache.lucene.search;
/*
* 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.
*/
/**
* A cache for filters.
*
* @lucene.experimental
*/
public interface FilterCache {
/**
* Return a wrapper around the provided <code>filter</code> that will cache
* {@link DocIdSet}s per-segment accordingly to the given <code>policy</code>.
*/
Filter doCache(Filter filter, FilterCachingPolicy policy);
}

View File

@ -0,0 +1,105 @@
package org.apache.lucene.search;
/*
* 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.
*/
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.
*
* Implementations of this class must be thread-safe.
*
* @lucene.experimental
*/
// TODO: add APIs for integration with IndexWriter.IndexReaderWarmer
public interface FilterCachingPolicy {
/** A simple policy that caches all the provided filters on all segments. */
public static final FilterCachingPolicy ALWAYS_CACHE = new FilterCachingPolicy() {
@Override
public void onCache(Filter filter) {}
@Override
public boolean shouldCache(Filter filter, LeafReaderContext context, DocIdSet set) throws IOException {
return true;
}
};
/** 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 FilterCachingPolicy {
/** {@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 LRUFilterCache}
* perform better when the number of cached segments is low). */
public static final CacheOnLargeSegments DEFAULT = new CacheOnLargeSegments(0.03f);
private final float minSizeRatio;
/**
* Create a {@link CacheOnLargeSegments} instance that only caches on a
* given segment if its number of documents divided by the total number of
* documents in the index is greater than or equal to
* <code>minSizeRatio</code>.
*/
public CacheOnLargeSegments(float minSizeRatio) {
if (minSizeRatio <= 0 || minSizeRatio >= 1) {
throw new IllegalArgumentException("minSizeRatio must be in ]0, 1[, got " + minSizeRatio);
}
this.minSizeRatio = minSizeRatio;
}
@Override
public void onCache(Filter filter) {}
@Override
public boolean shouldCache(Filter filter, LeafReaderContext context, DocIdSet set) throws IOException {
final IndexReaderContext topLevelContext = ReaderUtil.getTopLevelContext(context);
final float sizeRatio = (float) context.reader().maxDoc() / topLevelContext.reader().maxDoc();
return sizeRatio >= minSizeRatio;
}
};
/** Callback that is called on every call to {@link FilterCache#doCache}.
* This is typically useful if the policy wants to track usage statistics
* in order to make decisions. */
void onCache(Filter filter);
/** Whether the given {@link DocIdSet} should be cached on a given segment.
* This method will be called on each leaf context to know if the filter
* should be cached on this particular leaf. The filter cache will first
* attempt to load a {@link DocIdSet} from the cache. If it is not cached
* yet and this method returns <tt>true</tt> then a cache entry will be
* generated. Otherwise an uncached set will be returned. */
boolean shouldCache(Filter filter, LeafReaderContext context, DocIdSet set) throws IOException;
}

View File

@ -0,0 +1,338 @@
package org.apache.lucene.search;
/*
* 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.
*/
import java.io.IOException;
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.Map;
import java.util.Set;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReader.CoreClosedListener;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.RoaringDocIdSet;
/**
* A {@link FilterCache} that evicts filters using a LRU (least-recently-used)
* eviction policy in order to remain under a given maximum size and number of
* bytes used.
*
* This class is thread-safe.
*
* Note that filter eviction runs in linear time with the total number of
* segments that have cache entries so this cache works best with
* {@link FilterCachingPolicy caching policies} that only cache on "large"
* segments, and it is advised to not share this cache across too many indices.
*
* @lucene.experimental
*/
public class LRUFilterCache implements FilterCache, Accountable {
private final int maxSize;
private final long maxRamBytesUsed;
// 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
private final Set<Filter> mostRecentlyUsedFilters;
private final Map<Object, LeafCache> cache;
private volatile long ramBytesUsed; // all updates of this number must be performed under a lock
/**
* Create a new instance that will cache at most <code>maxSize</code> filters
* with at most <code>maxRamBytesUsed</code> bytes of memory.
*/
public LRUFilterCache(int maxSize, long maxRamBytesUsed) {
this.maxSize = maxSize;
this.maxRamBytesUsed = maxRamBytesUsed;
mostRecentlyUsedFilters = Collections.newSetFromMap(new LinkedHashMap<Filter, Boolean>(16, 0.75f, true));
cache = new IdentityHashMap<>();
ramBytesUsed = 0;
}
/** Whether evictions are required. */
boolean requiresEviction() {
return mostRecentlyUsedFilters.size() > maxSize || ramBytesUsed() > maxRamBytesUsed;
}
synchronized DocIdSet get(Filter filter, LeafReaderContext context) {
final LeafCache leafCache = cache.get(context.reader().getCoreCacheKey());
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;
}
return set;
}
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);
}
LeafCache leafCache = cache.get(context.reader().getCoreCacheKey());
if (leafCache == null) {
leafCache = new LeafCache();
final LeafCache previous = cache.put(context.reader().getCoreCacheKey(), leafCache);
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(filter, set);
evictIfNecessary();
}
synchronized void evictIfNecessary() {
// under a lock to make sure that mostRecentlyUsedFilters and cache keep sync'ed
if (requiresEviction() && mostRecentlyUsedFilters.isEmpty() == false) {
Iterator<Filter> iterator = mostRecentlyUsedFilters.iterator();
do {
final Filter filter = iterator.next();
iterator.remove();
ramBytesUsed -= ramBytesUsed(filter);
clearFilter(filter);
} while (iterator.hasNext() && requiresEviction());
}
}
/**
* 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 -= leafCache.ramBytesUsed;
}
}
/**
* Remove all cache entries for the given filter.
*/
public synchronized void clearFilter(Filter filter) {
for (LeafCache leafCache : cache.values()) {
leafCache.remove(filter);
}
}
/**
* Clear the content of this cache.
*/
public synchronized void clear() {
cache.clear();
mostRecentlyUsedFilters.clear();
ramBytesUsed = 0;
}
// pkg-private for testing
synchronized void assertConsistent() {
if (requiresEviction()) {
throw new AssertionError("requires evictions: size=" + mostRecentlyUsedFilters.size()
+ ", maxSize=" + maxSize + ", ramBytesUsed=" + ramBytesUsed() + ", maxRamBytesUsed=" + maxRamBytesUsed);
}
for (LeafCache leafCache : cache.values()) {
Set<Filter> keys = new HashSet<Filter>(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;
for (Filter filter : mostRecentlyUsedFilters) {
recomputedRamBytesUsed += ramBytesUsed(filter);
}
for (LeafCache leafCache : cache.values()) {
recomputedRamBytesUsed += leafCache.ramBytesUsed();
}
if (recomputedRamBytesUsed != ramBytesUsed) {
throw new AssertionError("ramBytesUsed mismatch : " + ramBytesUsed + " != " + recomputedRamBytesUsed);
}
}
// pkg-private for testing
synchronized Set<Filter> cachedFilters() {
return new HashSet<>(mostRecentlyUsedFilters);
}
@Override
public Filter doCache(Filter filter, FilterCachingPolicy policy) {
while (filter instanceof CachingWrapperFilter) {
// should we throw an exception instead?
filter = ((CachingWrapperFilter) filter).in;
}
return new CachingWrapperFilter(filter, policy);
}
/**
* Provide the DocIdSet to be cached, using the DocIdSet provided
* by the wrapped Filter. <p>This implementation returns the given {@link DocIdSet},
* if {@link DocIdSet#isCacheable} returns <code>true</code>, else it calls
* {@link #cacheImpl(DocIdSetIterator, org.apache.lucene.index.LeafReader)}
* <p>Note: This method returns {@linkplain DocIdSet#EMPTY} if the given docIdSet
* is <code>null</code> or if {@link DocIdSet#iterator()} return <code>null</code>. The empty
* instance is use as a placeholder in the cache instead of the <code>null</code> value.
*/
protected DocIdSet docIdSetToCache(DocIdSet docIdSet, LeafReader reader) throws IOException {
if (docIdSet == null || docIdSet.isCacheable()) {
return docIdSet;
} else {
final DocIdSetIterator it = docIdSet.iterator();
if (it == null) {
return null;
} else {
return cacheImpl(it, reader);
}
}
}
@Override
public long ramBytesUsed() {
return ramBytesUsed;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
synchronized (this) {
return Accountables.namedAccountables("segment", cache);
}
}
/**
* Return the number of bytes used by the given filter. The default
* implementation returns {@link Accountable#ramBytesUsed()} if the filter
* implements {@link Accountable} and <code>1024</code> otherwise.
*/
protected long ramBytesUsed(Filter filter) {
if (filter instanceof Accountable) {
return ((Accountable) filter).ramBytesUsed();
}
return 1024;
}
/**
* Default cache implementation: uses {@link RoaringDocIdSet}.
*/
protected DocIdSet cacheImpl(DocIdSetIterator iterator, LeafReader reader) throws IOException {
return new RoaringDocIdSet.Builder(reader.maxDoc()).add(iterator).build();
}
// this class is not thread-safe, everything but ramBytesUsed needs to be called under a lock
private class LeafCache implements Accountable {
private final Map<Filter, DocIdSet> cache;
private volatile long ramBytesUsed;
LeafCache() {
cache = new HashMap<>();
ramBytesUsed = 0;
}
private void incrementRamBytesUsed(long inc) {
ramBytesUsed += inc;
LRUFilterCache.this.ramBytesUsed += inc;
}
DocIdSet get(Filter filter) {
return cache.get(filter);
}
void putIfAbsent(Filter filter, DocIdSet set) {
if (cache.putIfAbsent(filter, set) == null) {
// the set was actually put
incrementRamBytesUsed(set.ramBytesUsed());
}
}
void remove(Filter filter) {
DocIdSet removed = cache.remove(filter);
if (removed != null) {
incrementRamBytesUsed(-removed.ramBytesUsed());
}
}
@Override
public long ramBytesUsed() {
return ramBytesUsed;
}
}
private class CachingWrapperFilter extends Filter {
private final Filter in;
private final FilterCachingPolicy policy;
CachingWrapperFilter(Filter in, FilterCachingPolicy policy) {
this.in = in;
this.policy = policy;
}
@Override
public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
DocIdSet set = get(in, context);
if (set == null) {
// do not apply acceptDocs yet, we want the cached filter to not take them into account
set = in.getDocIdSet(context, null);
if (policy.shouldCache(in, context, set)) {
set = docIdSetToCache(set, context.reader());
if (set == null) {
// null values are not supported
set = DocIdSet.EMPTY;
}
// it might happen that another thread computed the same set in parallel
// although this might incur some CPU overhead, it is probably better
// this way than trying to lock and preventing other filters to be
// computed at the same time?
putIfAbsent(in, context, set);
}
}
return set == DocIdSet.EMPTY ? null : BitsFilteredDocIdSet.wrap(set, acceptDocs);
}
@Override
public boolean equals(Object obj) {
return obj instanceof CachingWrapperFilter
&& in.equals(((CachingWrapperFilter) obj).in);
}
@Override
public int hashCode() {
return in.hashCode() ^ getClass().hashCode();
}
}
}

View File

@ -0,0 +1,129 @@
package org.apache.lucene.search;
/*
* 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.
*/
import java.io.IOException;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.FrequencyTrackingRingBuffer;
/**
* A {@link FilterCachingPolicy} that tracks usage statistics of recently-used
* filters in order to decide on which filters are worth caching.
*
* It also uses some heuristics on segments, filters and the doc id sets that
* they produce in order to cache more aggressively when the execution cost
* significantly outweighs the caching overhead.
*
* @lucene.experimental
*/
public final class UsageTrackingFilterCachingPolicy implements FilterCachingPolicy {
private static boolean isCostly(Filter filter) {
// This does not measure the cost of iterating over the filter (for this we
// already have the DocIdSetIterator#cost API) but the cost to build the
// DocIdSet in the first place
return filter instanceof MultiTermQueryWrapperFilter;
}
private static boolean isCheapToCache(DocIdSet set) {
// the produced doc set is already cacheable, so caching has no
// overhead at all. TODO: extend this to sets whose iterators have a low
// cost?
return set.isCacheable();
}
private final FilterCachingPolicy.CacheOnLargeSegments segmentPolicy;
private final FrequencyTrackingRingBuffer<Integer> recentlyUsedFilters;
private final int minFrequencyCostlyFilters;
private final int minFrequencyCheapFilters;
private final int minFrequencyOtherFilters;
/**
* Create a new instance.
*
* @param minSizeRatio the minimum size ratio for segments to be cached, see {@link FilterCachingPolicy.CacheOnLargeSegments}
* @param historySize the number of recently used filters to track
* @param minFrequencyCostlyFilters how many times filters whose {@link Filter#getDocIdSet(LeafReaderContext, Bits) getDocIdSet} method is expensive should have been seen before being cached
* @param minFrequencyCheapFilters how many times filters that produce {@link DocIdSet}s that are cheap to cached should have been seen before being cached
* @param minFrequencyOtherFilters how many times other filters should have been seen before being cached
*/
public UsageTrackingFilterCachingPolicy(
float minSizeRatio,
int historySize,
int minFrequencyCostlyFilters,
int minFrequencyCheapFilters,
int minFrequencyOtherFilters) {
this(new FilterCachingPolicy.CacheOnLargeSegments(minSizeRatio), historySize,
minFrequencyCostlyFilters, minFrequencyCheapFilters, minFrequencyOtherFilters);
}
/** Create a new instance with sensible defaults. */
public UsageTrackingFilterCachingPolicy() {
// we track the most 256 recently-used filters and cache filters that are
// expensive to build or cheap to cache after we have seen them twice, and
// cache regular filters after we have seen them 5 times
this(FilterCachingPolicy.CacheOnLargeSegments.DEFAULT, 256, 2, 2, 5);
}
private UsageTrackingFilterCachingPolicy(
FilterCachingPolicy.CacheOnLargeSegments segmentPolicy,
int historySize,
int minFrequencyCostlyFilters,
int minFrequencyCheapFilters,
int minFrequencyOtherFilters) {
this.segmentPolicy = segmentPolicy;
if (minFrequencyOtherFilters < minFrequencyCheapFilters || minFrequencyOtherFilters < minFrequencyCheapFilters) {
throw new IllegalArgumentException("it does not make sense to cache regular filters more aggressively than filters that are costly to produce or cheap to cache");
}
if (minFrequencyCheapFilters > historySize || minFrequencyCostlyFilters > historySize || minFrequencyOtherFilters > historySize) {
throw new IllegalArgumentException("The minimum frequencies should be less than the size of the history of filters that are being tracked");
}
this.recentlyUsedFilters = new FrequencyTrackingRingBuffer<>(historySize);
this.minFrequencyCostlyFilters = minFrequencyCostlyFilters;
this.minFrequencyCheapFilters = minFrequencyCheapFilters;
this.minFrequencyOtherFilters = minFrequencyOtherFilters;
}
@Override
public void onCache(Filter filter) {
// Using the filter hash codes might help keep memory usage a bit lower
// since some filters might have non-negligible memory usage?
recentlyUsedFilters.add(filter.hashCode());
}
@Override
public boolean shouldCache(Filter filter, LeafReaderContext context, DocIdSet set) throws IOException {
if (segmentPolicy.shouldCache(filter, context, set) == false) {
return false;
}
final int frequency = recentlyUsedFilters.frequency(filter.hashCode());
if (frequency >= minFrequencyOtherFilters) {
return true;
} else if (isCostly(filter) && frequency >= minFrequencyCostlyFilters) {
return true;
} else if (isCheapToCache(set) && frequency >= minFrequencyCheapFilters) {
return true;
}
return false;
}
}

View File

@ -0,0 +1,91 @@
package org.apache.lucene.util;
/*
* 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.
*/
import java.util.ArrayDeque;
import java.util.Collections;
import java.util.Deque;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
/**
* A ring buffer that tracks the frequency of the items that it contains.
* This is typically useful to track popular recently-used items.
*
* This class is thread-safe.
*
* @lucene.internal
*/
public final class FrequencyTrackingRingBuffer<T> {
private final int maxSize;
private final Deque<T> ringBuffer;
private final ConcurrentMap<T, Integer> frequencies;
/** Create a new ring buffer that will contain at most <code>size</code> items. */
public FrequencyTrackingRingBuffer(int maxSize) {
this.maxSize = maxSize;
this.ringBuffer = new ArrayDeque<>(maxSize);
this.frequencies = new ConcurrentHashMap<>();
}
/**
* Add a new item to this ring buffer, potentially removing the oldest
* entry from this buffer if it is already full.
*/
public synchronized void add(T item) {
// we need this method to be protected by a lock since it is important for
// correctness that the ring buffer and the frequencies table have
// consistent content
if (item == null) {
throw new IllegalArgumentException("null items are not supported");
}
assert ringBuffer.size() <= maxSize;
if (ringBuffer.size() == maxSize) {
// evict the oldest entry
final T removed = ringBuffer.removeFirst();
final int newFrequency = frequency(removed) - 1;
if (newFrequency == 0) {
// free for GC
frequencies.remove(removed);
} else {
frequencies.put(removed, newFrequency);
}
}
// add the new entry and update frequencies
ringBuffer.addLast(item);
frequencies.put(item, frequency(item) + 1);
}
/**
* Returns the frequency of the provided item in the ring buffer.
*/
public int frequency(T item) {
// The use of a concurrent hash map allows us to not use a lock for this read-only method
final Integer freq = frequencies.get(item);
return freq == null ? 0 : freq;
}
// pkg-private for testing
Map<T, Integer> asFrequencyMap() {
return Collections.unmodifiableMap(frequencies);
}
}

View File

@ -89,7 +89,7 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
public void testEmpty() throws Exception {
Query query = new BooleanQuery();
Filter expected = new QueryWrapperFilter(query);
Filter actual = new CachingWrapperFilter(expected);
Filter actual = new CachingWrapperFilter(expected, MAYBE_CACHE_POLICY);
assertFilterEquals(expected, actual);
}
@ -99,14 +99,14 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
query.add(new TermQuery(new Term("id", "0")), BooleanClause.Occur.MUST);
query.add(new TermQuery(new Term("id", "0")), BooleanClause.Occur.MUST_NOT);
Filter expected = new QueryWrapperFilter(query);
Filter actual = new CachingWrapperFilter(expected);
Filter actual = new CachingWrapperFilter(expected, MAYBE_CACHE_POLICY);
assertFilterEquals(expected, actual);
}
/** test null docidset */
public void testEmpty3() throws Exception {
Filter expected = new PrefixFilter(new Term("bogusField", "bogusVal"));
Filter actual = new CachingWrapperFilter(expected);
Filter actual = new CachingWrapperFilter(expected, MAYBE_CACHE_POLICY);
assertFilterEquals(expected, actual);
}
@ -116,7 +116,7 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
int id = random().nextInt(ir.maxDoc());
Query query = new TermQuery(new Term("id", Integer.toString(id)));
Filter expected = new QueryWrapperFilter(query);
Filter actual = new CachingWrapperFilter(expected);
Filter actual = new CachingWrapperFilter(expected, MAYBE_CACHE_POLICY);
assertFilterEquals(expected, actual);
}
}
@ -129,7 +129,7 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
Query query = TermRangeQuery.newStringRange("id",
Integer.toString(id_start), Integer.toString(id_end), true, true);
Filter expected = new QueryWrapperFilter(query);
Filter actual = new CachingWrapperFilter(expected);
Filter actual = new CachingWrapperFilter(expected, MAYBE_CACHE_POLICY);
assertFilterEquals(expected, actual);
}
}
@ -138,7 +138,7 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
public void testDense() throws Exception {
Query query = new MatchAllDocsQuery();
Filter expected = new QueryWrapperFilter(query);
Filter actual = new CachingWrapperFilter(expected);
Filter actual = new CachingWrapperFilter(expected, FilterCachingPolicy.ALWAYS_CACHE);
assertFilterEquals(expected, actual);
}
@ -150,7 +150,7 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
IndexReader reader = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(dir));
LeafReaderContext context = (LeafReaderContext) reader.getContext();
MockFilter filter = new MockFilter();
CachingWrapperFilter cacher = new CachingWrapperFilter(filter);
CachingWrapperFilter cacher = new CachingWrapperFilter(filter, FilterCachingPolicy.ALWAYS_CACHE);
// first time, nested filter is called
DocIdSet strongRef = cacher.getDocIdSet(context, context.reader().getLiveDocs());
@ -182,7 +182,7 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
return null;
}
};
CachingWrapperFilter cacher = new CachingWrapperFilter(filter);
CachingWrapperFilter cacher = new CachingWrapperFilter(filter, MAYBE_CACHE_POLICY);
// the caching filter should return the empty set constant
assertNull(cacher.getDocIdSet(context, context.reader().getLiveDocs()));
@ -215,7 +215,7 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
};
}
};
CachingWrapperFilter cacher = new CachingWrapperFilter(filter);
CachingWrapperFilter cacher = new CachingWrapperFilter(filter, FilterCachingPolicy.ALWAYS_CACHE);
// the caching filter should return the empty set constant
assertNull(cacher.getDocIdSet(context, context.reader().getLiveDocs()));
@ -227,7 +227,7 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
private static void assertDocIdSetCacheable(IndexReader reader, Filter filter, boolean shouldCacheable) throws IOException {
assertTrue(reader.getContext() instanceof LeafReaderContext);
LeafReaderContext context = (LeafReaderContext) reader.getContext();
final CachingWrapperFilter cacher = new CachingWrapperFilter(filter);
final CachingWrapperFilter cacher = new CachingWrapperFilter(filter, FilterCachingPolicy.ALWAYS_CACHE);
final DocIdSet originalSet = filter.getDocIdSet(context, context.reader().getLiveDocs());
final DocIdSet cachedSet = cacher.getDocIdSet(context, context.reader().getLiveDocs());
if (originalSet == null) {
@ -305,7 +305,7 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
final Filter startFilter = new QueryWrapperFilter(new TermQuery(new Term("id", "1")));
CachingWrapperFilter filter = new CachingWrapperFilter(startFilter);
CachingWrapperFilter filter = new CachingWrapperFilter(startFilter, FilterCachingPolicy.ALWAYS_CACHE);
docs = searcher.search(new MatchAllDocsQuery(), filter, 1);
assertTrue(filter.ramBytesUsed() > 0);
@ -353,7 +353,7 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
assertEquals("[just filter] Should *not* find a hit...", 0, docs.totalHits);
// apply deletes dynamically:
filter = new CachingWrapperFilter(startFilter);
filter = new CachingWrapperFilter(startFilter, FilterCachingPolicy.ALWAYS_CACHE);
writer.addDocument(doc);
reader = refreshReader(reader);
searcher = newSearcher(reader, false);

View File

@ -0,0 +1,186 @@
package org.apache.lucene.search;
/*
* 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.
*/
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field.Store;
import org.apache.lucene.document.StringField;
import org.apache.lucene.index.DirectoryReader;
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;
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
public class TestLRUFilterCache extends LuceneTestCase {
private static final FilterCachingPolicy NEVER_CACHE = new FilterCachingPolicy() {
@Override
public void onCache(Filter filter) {}
@Override
public boolean shouldCache(Filter filter, LeafReaderContext context, DocIdSet set) throws IOException {
return false;
}
};
public void testConcurrency() throws Throwable {
final LRUFilterCache filterCache = new LRUFilterCache(1 + random().nextInt(20), 1 + random().nextInt(10000));
Directory dir = newDirectory();
final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
SearcherManager mgr = new SearcherManager(w.w, random().nextBoolean(), new SearcherFactory());
final AtomicBoolean indexing = new AtomicBoolean(true);
final AtomicReference<Throwable> error = new AtomicReference<>();
final int numDocs = atLeast(10000);
Thread[] threads = new Thread[3];
threads[0] = new Thread() {
public void run() {
Document doc = new Document();
StringField f = new StringField("color", "", Store.NO);
doc.add(f);
for (int i = 0; indexing.get() && i < numDocs; ++i) {
f.setStringValue(RandomPicks.randomFrom(random(), new String[] {"blue", "red", "yellow"}));
try {
w.addDocument(doc);
if ((i & 63) == 0) {
mgr.maybeRefresh();
if (rarely()) {
filterCache.clear();
}
if (rarely()) {
final String color = RandomPicks.randomFrom(random(), new String[] {"blue", "red", "yellow"});
w.deleteDocuments(new Term("color", color));
}
}
} catch (Throwable t) {
error.compareAndSet(null, t);
break;
}
}
indexing.set(false);
}
};
for (int i = 1; i < threads.length; ++i) {
threads[i] = new Thread() {
@Override
public void run() {
while (indexing.get()) {
try {
final IndexSearcher searcher = mgr.acquire();
try {
final String value = RandomPicks.randomFrom(random(), new String[] {"blue", "red", "yellow", "green"});
final Filter f = new QueryWrapperFilter(new TermQuery(new Term("color", value)));
final Filter cached = filterCache.doCache(f, MAYBE_CACHE_POLICY);
TotalHitCountCollector collector = new TotalHitCountCollector();
searcher.search(new ConstantScoreQuery(cached), collector);
TotalHitCountCollector collector2 = new TotalHitCountCollector();
searcher.search(new ConstantScoreQuery(f), collector2);
assertEquals(collector.getTotalHits(), collector2.getTotalHits());
} finally {
mgr.release(searcher);
}
} catch (Throwable t) {
error.compareAndSet(null, t);
}
}
}
};
}
for (Thread thread : threads) {
thread.start();
}
for (Thread thread : threads) {
thread.join();
}
if (error.get() != null) {
throw error.get();
}
filterCache.assertConsistent();
mgr.close();
w.close();
dir.close();
filterCache.assertConsistent();
}
public void testLRUEviction() throws Exception {
Directory dir = newDirectory();
final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
Document doc = new Document();
StringField f = new StringField("color", "blue", Store.NO);
doc.add(f);
w.addDocument(doc);
f.setStringValue("red");
w.addDocument(doc);
f.setStringValue("green");
w.addDocument(doc);
final DirectoryReader reader = w.getReader();
final IndexSearcher searcher = newSearcher(reader);
final LRUFilterCache filterCache = new LRUFilterCache(2, 100000);
final Filter blue = new QueryWrapperFilter(new TermQuery(new Term("color", "blue")));
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());
// 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());
searcher.search(new ConstantScoreQuery(filterCache.doCache(red, FilterCachingPolicy.ALWAYS_CACHE)), 1);
assertEquals(Collections.singleton(red), filterCache.cachedFilters());
searcher.search(new ConstantScoreQuery(filterCache.doCache(green, FilterCachingPolicy.ALWAYS_CACHE)), 1);
assertEquals(new HashSet<>(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());
searcher.search(new ConstantScoreQuery(filterCache.doCache(blue, FilterCachingPolicy.ALWAYS_CACHE)), 1);
assertEquals(new HashSet<>(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());
searcher.search(new ConstantScoreQuery(filterCache.doCache(green, FilterCachingPolicy.ALWAYS_CACHE)), 1);
assertEquals(new HashSet<>(Arrays.asList(green, blue)), filterCache.cachedFilters());
searcher.search(new ConstantScoreQuery(filterCache.doCache(red, NEVER_CACHE)), 1);
assertEquals(new HashSet<>(Arrays.asList(green, blue)), filterCache.cachedFilters());
reader.close();
w.close();
dir.close();
}
}

View File

@ -0,0 +1,63 @@
package org.apache.lucene.util;
/*
* 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.
*/
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
public class TestFrequencyTrackingRingBuffer extends LuceneTestCase {
private static <T> void assertBuffer(FrequencyTrackingRingBuffer<T> buffer, int maxSize, List<T> items) {
final List<T> recentItems;
if (items.size() <= maxSize) {
recentItems = items;
} else {
recentItems = items.subList(items.size() - maxSize, items.size());
}
final Map<T, Integer> expectedFrequencies = new HashMap<T, Integer>();
for (T item : recentItems) {
final Integer freq = expectedFrequencies.get(item);
if (freq == null) {
expectedFrequencies.put(item, 1);
} else {
expectedFrequencies.put(item, freq + 1);
}
}
assertEquals(expectedFrequencies, buffer.asFrequencyMap());
}
public void test() {
final int iterations = atLeast(100);
for (int i = 0; i < iterations; ++i) {
final int maxSize = 1 + random().nextInt(100);
final int numitems = random().nextInt(500);
final int maxitem = 1 + random().nextInt(100);
List<Integer> items = new ArrayList<>();
FrequencyTrackingRingBuffer<Integer> buffer = new FrequencyTrackingRingBuffer<>(maxSize);
for (int j = 0; j < numitems; ++j) {
final Integer item = random().nextInt(maxitem);
items.add(item);
buffer.add(item);
}
assertBuffer(buffer, maxSize, items);
}
}
}

View File

@ -59,6 +59,7 @@ import org.apache.lucene.search.CachingWrapperFilter;
import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.FilterCachingPolicy;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.NumericRangeFilter;
@ -920,7 +921,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
final AtomicBoolean filterWasUsed = new AtomicBoolean();
if (random().nextBoolean()) {
// Sort of silly:
fastMatchFilter = new CachingWrapperFilter(new QueryWrapperFilter(new MatchAllDocsQuery())) {
fastMatchFilter = new CachingWrapperFilter(new QueryWrapperFilter(new MatchAllDocsQuery()), FilterCachingPolicy.ALWAYS_CACHE) {
@Override
protected DocIdSet cacheImpl(DocIdSetIterator iterator, LeafReader reader)
throws IOException {

View File

@ -25,6 +25,7 @@ import org.apache.lucene.search.CachingWrapperFilter;
import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.FilterCachingPolicy;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.BitDocIdSet;
@ -39,7 +40,7 @@ public class BitDocIdSetCachingWrapperFilter extends BitDocIdSetFilter implement
/** Sole constructor. */
public BitDocIdSetCachingWrapperFilter(Filter filter) {
super();
this.filter = new CachingWrapperFilter(filter) {
this.filter = new CachingWrapperFilter(filter, FilterCachingPolicy.ALWAYS_CACHE) {
@Override
protected BitDocIdSet docIdSetToCache(DocIdSet docIdSet, LeafReader reader) throws IOException {
if (docIdSet == null || docIdSet instanceof BitDocIdSet) {

View File

@ -36,6 +36,7 @@ import org.apache.lucene.search.CachingWrapperFilter;
import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.FilterCachingPolicy;
import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
@ -50,7 +51,7 @@ public class TestBlockJoinSorter extends LuceneTestCase {
private static class FixedBitSetCachingWrapperFilter extends CachingWrapperFilter {
public FixedBitSetCachingWrapperFilter(Filter filter) {
super(filter);
super(filter, FilterCachingPolicy.ALWAYS_CACHE);
}
@Override

View File

@ -57,7 +57,6 @@ import java.util.logging.Logger;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field.Store;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
@ -110,7 +109,10 @@ import org.apache.lucene.index.TermsEnum.SeekStatus;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.TieredMergePolicy;
import org.apache.lucene.search.AssertingIndexSearcher;
import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.FilterCachingPolicy;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.QueryUtils.FCInvisibleMultiReader;
import org.apache.lucene.store.BaseDirectoryWrapper;
@ -140,6 +142,7 @@ import org.junit.Test;
import org.junit.rules.RuleChain;
import org.junit.rules.TestRule;
import org.junit.runner.RunWith;
import com.carrotsearch.randomizedtesting.JUnit4MethodProvider;
import com.carrotsearch.randomizedtesting.LifecycleScope;
import com.carrotsearch.randomizedtesting.MixWithSuiteName;
@ -467,6 +470,19 @@ public abstract class LuceneTestCase extends Assert {
CORE_DIRECTORIES.add("RAMDirectory");
};
/** A {@link FilterCachingPolicy} that randomly caches. */
public static final FilterCachingPolicy MAYBE_CACHE_POLICY = new FilterCachingPolicy() {
@Override
public void onCache(Filter filter) {}
@Override
public boolean shouldCache(Filter filter, LeafReaderContext context, DocIdSet set) throws IOException {
return random().nextBoolean();
}
};
// -----------------------------------------------------------------
// Fields initialized in class or instance rules.
// -----------------------------------------------------------------