mirror of https://github.com/apache/lucene.git
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:
parent
c539640102
commit
86a6c37b47
|
@ -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
|
||||
|
|
|
@ -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,14 +108,17 @@ public class CachingWrapperFilter extends Filter implements Accountable {
|
|||
if (docIdSet != null) {
|
||||
hitCount++;
|
||||
} else {
|
||||
missCount++;
|
||||
docIdSet = docIdSetToCache(filter.getDocIdSet(context, null), reader);
|
||||
if (docIdSet == null) {
|
||||
// We use EMPTY as a sentinel for the empty set, which is cacheable
|
||||
docIdSet = EMPTY;
|
||||
docIdSet = filter.getDocIdSet(context, null);
|
||||
if (policy.shouldCache(filter, context, docIdSet)) {
|
||||
missCount++;
|
||||
docIdSet = docIdSetToCache(docIdSet, reader);
|
||||
if (docIdSet == null) {
|
||||
// We use EMPTY as a sentinel for the empty set, which is cacheable
|
||||
docIdSet = EMPTY;
|
||||
}
|
||||
assert docIdSet.isCacheable();
|
||||
cache.put(key, docIdSet);
|
||||
}
|
||||
assert docIdSet.isCacheable();
|
||||
cache.put(key, docIdSet);
|
||||
}
|
||||
|
||||
return docIdSet == EMPTY ? null : BitsFilteredDocIdSet.wrap(docIdSet, acceptDocs);
|
||||
|
|
|
@ -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);
|
||||
|
||||
}
|
|
@ -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;
|
||||
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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 {
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
@ -466,7 +469,20 @@ public abstract class LuceneTestCase extends Assert {
|
|||
CORE_DIRECTORIES = new ArrayList<>(FS_DIRECTORIES);
|
||||
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.
|
||||
// -----------------------------------------------------------------
|
||||
|
|
Loading…
Reference in New Issue