mirror of https://github.com/apache/lucene.git
LUCENE-7410: Make cache keys and close listeners less trappy.
This commit is contained in:
parent
8e65aca0e1
commit
df6f830723
|
@ -33,6 +33,10 @@ API Changes
|
|||
|
||||
* LUCENE-7494: Points now have a per-field API, like doc values. (Adrien Grand)
|
||||
|
||||
* LUCENE-7410: Cache keys and close listeners have been refactored in order
|
||||
to be less trappy. See IndexReader.getReaderCacheHelper and
|
||||
LeafReader.getCoreCacheHelper. (Adrien Grand)
|
||||
|
||||
Bug Fixes
|
||||
|
||||
* LUCENE-7626: IndexWriter will no longer accept broken token offsets
|
||||
|
|
|
@ -47,3 +47,11 @@ queries.
|
|||
|
||||
This option has been removed as expanded terms are now normalized through
|
||||
Analyzer#normalize.
|
||||
|
||||
## Cache key and close listener refactoring (LUCENE-7410)
|
||||
|
||||
The way to access cache keys and add close listeners has been refactored in
|
||||
order to be less trappy. You should now use IndexReader.getReaderCacheHelper()
|
||||
to have manage caches that take deleted docs and doc values updates into
|
||||
account, and LeafReader.getCoreCacheHelper() to manage per-segment caches that
|
||||
do not take deleted docs and doc values updates into account.
|
||||
|
|
|
@ -114,6 +114,16 @@ public class FixBrokenOffsets {
|
|||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
|
|
|
@ -521,7 +521,7 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
}
|
||||
|
||||
// step 2: create ordinal map (this conceptually does the "merging")
|
||||
final OrdinalMap map = OrdinalMap.build(this, liveTerms, weights, PackedInts.COMPACT);
|
||||
final OrdinalMap map = OrdinalMap.build(null, liveTerms, weights, PackedInts.COMPACT);
|
||||
|
||||
// step 3: add field
|
||||
addSortedField(fieldInfo,
|
||||
|
@ -689,7 +689,7 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
}
|
||||
|
||||
// step 2: create ordinal map (this conceptually does the "merging")
|
||||
final OrdinalMap map = OrdinalMap.build(this, liveTerms, weights, PackedInts.COMPACT);
|
||||
final OrdinalMap map = OrdinalMap.build(null, liveTerms, weights, PackedInts.COMPACT);
|
||||
|
||||
// step 3: add field
|
||||
addSortedSetField(mergeFieldInfo,
|
||||
|
|
|
@ -89,14 +89,16 @@ public class ExitableDirectoryReader extends FilterDirectoryReader {
|
|||
}
|
||||
}
|
||||
|
||||
// this impl does not change deletes or data so we can delegate the
|
||||
// CacheHelpers
|
||||
@Override
|
||||
public Object getCoreCacheKey() {
|
||||
return in.getCoreCacheKey();
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return in.getReaderCacheHelper();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getCombinedCoreAndDeletesKey() {
|
||||
return in.getCombinedCoreAndDeletesKey();
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return in.getCoreCacheHelper();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -210,6 +212,11 @@ public class ExitableDirectoryReader extends FilterDirectoryReader {
|
|||
return new ExitableDirectoryReader(in, queryTimeout);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return in.getReaderCacheHelper();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "ExitableDirectoryReader(" + in.toString() + ")";
|
||||
|
|
|
@ -35,6 +35,9 @@ import org.apache.lucene.util.Bits;
|
|||
* A <code>FilterCodecReader</code> contains another CodecReader, which it
|
||||
* uses as its basic source of data, possibly transforming the data along the
|
||||
* way or providing additional functionality.
|
||||
* <p><b>NOTE</b>: If this {@link FilterCodecReader} does not change the
|
||||
* content the contained reader, you could consider delegating calls to
|
||||
* {@link #getCoreCacheHelper()} and {@link #getReaderCacheHelper()}.
|
||||
*/
|
||||
public abstract class FilterCodecReader extends CodecReader {
|
||||
/**
|
||||
|
@ -105,16 +108,6 @@ public abstract class FilterCodecReader extends CodecReader {
|
|||
return in.getIndexSort();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addCoreClosedListener(CoreClosedListener listener) {
|
||||
in.addCoreClosedListener(listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeCoreClosedListener(CoreClosedListener listener) {
|
||||
in.removeCoreClosedListener(listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doClose() throws IOException {
|
||||
in.doClose();
|
||||
|
|
|
@ -19,9 +19,7 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
import java.util.Objects;
|
||||
|
||||
import org.apache.lucene.search.QueryCache;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.util.AttributeSource;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
@ -38,12 +36,8 @@ import org.apache.lucene.util.BytesRef;
|
|||
* <p><b>NOTE</b>: If you override {@link #getLiveDocs()}, you will likely need
|
||||
* to override {@link #numDocs()} as well and vice-versa.
|
||||
* <p><b>NOTE</b>: If this {@link FilterLeafReader} does not change the
|
||||
* content the contained reader, you could consider overriding
|
||||
* {@link #getCoreCacheKey()} so that
|
||||
* {@link QueryCache} impls share the same entries for this atomic reader
|
||||
* and the wrapped one. {@link #getCombinedCoreAndDeletesKey()} could be
|
||||
* overridden as well if the {@link #getLiveDocs() live docs} are not changed
|
||||
* either.
|
||||
* content the contained reader, you could consider delegating calls to
|
||||
* {@link #getCoreCacheHelper()} and {@link #getReaderCacheHelper()}.
|
||||
*/
|
||||
public abstract class FilterLeafReader extends LeafReader {
|
||||
|
||||
|
@ -307,69 +301,6 @@ public abstract class FilterLeafReader extends LeafReader {
|
|||
in.registerParentReader(this);
|
||||
}
|
||||
|
||||
/**
|
||||
* A CoreClosedListener wrapper that adjusts the core cache key that
|
||||
* the wrapper is called with. This is useful if the core cache key
|
||||
* of a reader is different from the key of the wrapped reader.
|
||||
*/
|
||||
private static class CoreClosedListenerWrapper implements CoreClosedListener {
|
||||
|
||||
public static CoreClosedListener wrap(CoreClosedListener listener, Object thisCoreKey, Object inCoreKey) {
|
||||
if (thisCoreKey == inCoreKey) {
|
||||
// this reader has the same core cache key as its parent, nothing to do
|
||||
return listener;
|
||||
} else {
|
||||
// we don't have the same cache key as the wrapped reader, we need to wrap
|
||||
// the listener to call it with the correct cache key
|
||||
return new CoreClosedListenerWrapper(listener, thisCoreKey, inCoreKey);
|
||||
}
|
||||
}
|
||||
|
||||
private final CoreClosedListener in;
|
||||
private final Object thisCoreKey;
|
||||
private final Object inCoreKey;
|
||||
|
||||
private CoreClosedListenerWrapper(CoreClosedListener in, Object thisCoreKey, Object inCoreKey) {
|
||||
this.in = in;
|
||||
this.thisCoreKey = thisCoreKey;
|
||||
this.inCoreKey = inCoreKey;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onClose(Object ownerCoreCacheKey) throws IOException {
|
||||
assert inCoreKey == ownerCoreCacheKey;
|
||||
in.onClose(thisCoreKey);
|
||||
}
|
||||
|
||||
// NOTE: equals/hashcore are important for removeCoreClosedListener to work
|
||||
// correctly
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (obj == null || obj.getClass() != CoreClosedListenerWrapper.class) {
|
||||
return false;
|
||||
}
|
||||
CoreClosedListenerWrapper that = (CoreClosedListenerWrapper) obj;
|
||||
return in.equals(that.in) && thisCoreKey == that.thisCoreKey;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(getClass(), in, thisCoreKey);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addCoreClosedListener(final CoreClosedListener listener) {
|
||||
in.addCoreClosedListener(CoreClosedListenerWrapper.wrap(listener, getCoreCacheKey(), in.getCoreCacheKey()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeCoreClosedListener(CoreClosedListener listener) {
|
||||
in.removeCoreClosedListener(CoreClosedListenerWrapper.wrap(listener, getCoreCacheKey(), in.getCoreCacheKey()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getLiveDocs() {
|
||||
ensureOpen();
|
||||
|
|
|
@ -20,7 +20,6 @@ package org.apache.lucene.index;
|
|||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.WeakHashMap;
|
||||
|
@ -90,41 +89,47 @@ public abstract class IndexReader implements Closeable {
|
|||
}
|
||||
|
||||
/**
|
||||
* A custom listener that's invoked when the IndexReader
|
||||
* is closed.
|
||||
*
|
||||
* A utility class that gives hooks in order to help build a cache based on
|
||||
* the data that is contained in this index.
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public static interface ReaderClosedListener {
|
||||
/** Invoked when the {@link IndexReader} is closed. */
|
||||
public void onClose(IndexReader reader) throws IOException;
|
||||
public static interface CacheHelper {
|
||||
|
||||
/**
|
||||
* Get a key that the resource can be cached on. The given entry can be
|
||||
* compared using identity, ie. {@link Object#equals} is implemented as
|
||||
* {@code ==} and {@link Object#hashCode} is implemented as
|
||||
* {@link System#identityHashCode}.
|
||||
*/
|
||||
CacheKey getKey();
|
||||
|
||||
/**
|
||||
* Add a {@link ClosedListener} which will be called when the resource
|
||||
* guarded by {@link #getKey()} is closed.
|
||||
*/
|
||||
void addClosedListener(ClosedListener listener);
|
||||
|
||||
}
|
||||
|
||||
private final Set<ReaderClosedListener> readerClosedListeners =
|
||||
Collections.synchronizedSet(new LinkedHashSet<ReaderClosedListener>());
|
||||
/** A cache key identifying a resource that is being cached on. */
|
||||
public static final class CacheKey {
|
||||
CacheKey() {} // only instantiable by core impls
|
||||
}
|
||||
|
||||
/**
|
||||
* A listener that is called when a resource gets closed.
|
||||
* @lucene.experimental
|
||||
*/
|
||||
@FunctionalInterface
|
||||
public static interface ClosedListener {
|
||||
/** Invoked when the resource (segment core, or index reader) that is
|
||||
* being cached on is closed. */
|
||||
void onClose(CacheKey key) throws IOException;
|
||||
}
|
||||
|
||||
private final Set<IndexReader> parentReaders =
|
||||
Collections.synchronizedSet(Collections.newSetFromMap(new WeakHashMap<IndexReader,Boolean>()));
|
||||
|
||||
/** Expert: adds a {@link ReaderClosedListener}. The
|
||||
* provided listener will be invoked when this reader is closed.
|
||||
* At this point, it is safe for apps to evict this reader from
|
||||
* any caches keyed on {@link #getCombinedCoreAndDeletesKey()}.
|
||||
*
|
||||
* @lucene.experimental */
|
||||
public final void addReaderClosedListener(ReaderClosedListener listener) {
|
||||
ensureOpen();
|
||||
readerClosedListeners.add(listener);
|
||||
}
|
||||
|
||||
/** Expert: remove a previously added {@link ReaderClosedListener}.
|
||||
*
|
||||
* @lucene.experimental */
|
||||
public final void removeReaderClosedListener(ReaderClosedListener listener) {
|
||||
ensureOpen();
|
||||
readerClosedListeners.remove(listener);
|
||||
}
|
||||
|
||||
/** Expert: This method is called by {@code IndexReader}s which wrap other readers
|
||||
* (e.g. {@link CompositeReader} or {@link FilterLeafReader}) to register the parent
|
||||
* at the child (this reader) on construction of the parent. When this reader is closed,
|
||||
|
@ -136,22 +141,11 @@ public abstract class IndexReader implements Closeable {
|
|||
parentReaders.add(reader);
|
||||
}
|
||||
|
||||
private void notifyReaderClosedListeners(Throwable th) throws IOException {
|
||||
synchronized(readerClosedListeners) {
|
||||
for(ReaderClosedListener listener : readerClosedListeners) {
|
||||
try {
|
||||
listener.onClose(this);
|
||||
} catch (Throwable t) {
|
||||
if (th == null) {
|
||||
th = t;
|
||||
} else {
|
||||
th.addSuppressed(t);
|
||||
}
|
||||
}
|
||||
}
|
||||
// overridden by StandardDirectoryReader and SegmentReader
|
||||
void notifyReaderClosedListeners(Throwable th) throws IOException {
|
||||
// nothing to notify in the base impl, just rethrow
|
||||
IOUtils.reThrow(th);
|
||||
}
|
||||
}
|
||||
|
||||
private void reportCloseToParentReaders() {
|
||||
synchronized(parentReaders) {
|
||||
|
@ -279,10 +273,8 @@ public abstract class IndexReader implements Closeable {
|
|||
}
|
||||
|
||||
/** {@inheritDoc}
|
||||
* <p>For caching purposes, {@code IndexReader} subclasses are not allowed
|
||||
* <p>{@code IndexReader} subclasses are not allowed
|
||||
* to implement equals/hashCode, so methods are declared final.
|
||||
* To lookup instances from caches use {@link #getCoreCacheKey} and
|
||||
* {@link #getCombinedCoreAndDeletesKey}.
|
||||
*/
|
||||
@Override
|
||||
public final boolean equals(Object obj) {
|
||||
|
@ -290,10 +282,8 @@ public abstract class IndexReader implements Closeable {
|
|||
}
|
||||
|
||||
/** {@inheritDoc}
|
||||
* <p>For caching purposes, {@code IndexReader} subclasses are not allowed
|
||||
* <p>{@code IndexReader} subclasses are not allowed
|
||||
* to implement equals/hashCode, so methods are declared final.
|
||||
* To lookup instances from caches use {@link #getCoreCacheKey} and
|
||||
* {@link #getCombinedCoreAndDeletesKey}.
|
||||
*/
|
||||
@Override
|
||||
public final int hashCode() {
|
||||
|
@ -436,23 +426,16 @@ public abstract class IndexReader implements Closeable {
|
|||
return getContext().leaves();
|
||||
}
|
||||
|
||||
/** Expert: Returns a key for this IndexReader, so CachingWrapperFilter can find
|
||||
* it again.
|
||||
* This key must not have equals()/hashCode() methods, so "equals" means "identical". */
|
||||
public Object getCoreCacheKey() {
|
||||
// Don't call ensureOpen since FC calls this (to evict)
|
||||
// on close
|
||||
return this;
|
||||
}
|
||||
|
||||
/** Expert: Returns a key for this IndexReader that also includes deletions,
|
||||
* so CachingWrapperFilter can find it again.
|
||||
* This key must not have equals()/hashCode() methods, so "equals" means "identical". */
|
||||
public Object getCombinedCoreAndDeletesKey() {
|
||||
// Don't call ensureOpen since FC calls this (to evict)
|
||||
// on close
|
||||
return this;
|
||||
}
|
||||
/**
|
||||
* Optional method: Return a {@link CacheHelper} that can be used to cache
|
||||
* based on the content of this reader. Two readers that have different data
|
||||
* or different sets of deleted documents will be considered different.
|
||||
* <p>A return value of {@code null} indicates that this reader is not suited
|
||||
* for caching, which is typically the case for short-lived wrappers that
|
||||
* alter the content of the wrapped reader.
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public abstract CacheHelper getReaderCacheHelper();
|
||||
|
||||
/** Returns the number of documents containing the
|
||||
* <code>term</code>. This method returns 0 if the term or
|
||||
|
|
|
@ -18,7 +18,7 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.IndexReader.ReaderClosedListener;
|
||||
import org.apache.lucene.index.IndexReader.CacheHelper;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
||||
|
@ -61,80 +61,18 @@ public abstract class LeafReader extends IndexReader {
|
|||
}
|
||||
|
||||
/**
|
||||
* Called when the shared core for this {@link LeafReader}
|
||||
* is closed.
|
||||
* <p>
|
||||
* If this {@link LeafReader} impl has the ability to share
|
||||
* resources across instances that might only vary through
|
||||
* deleted documents and doc values updates, then this listener
|
||||
* will only be called when the shared core is closed.
|
||||
* Otherwise, this listener will be called when this reader is
|
||||
* closed.</p>
|
||||
* <p>
|
||||
* This is typically useful to manage per-segment caches: when
|
||||
* the listener is called, it is safe to evict this reader from
|
||||
* any caches keyed on {@link #getCoreCacheKey}.</p>
|
||||
*
|
||||
* Optional method: Return a {@link CacheHelper} that can be used to cache
|
||||
* based on the content of this leaf regardless of deletions. Two readers
|
||||
* that have the same data but different sets of deleted documents or doc
|
||||
* values updates may be considered equal. Consider using
|
||||
* {@link #getReaderCacheHelper} if you need deletions or dv updates to be
|
||||
* taken into account.
|
||||
* <p>A return value of {@code null} indicates that this reader is not suited
|
||||
* for caching, which is typically the case for short-lived wrappers that
|
||||
* alter the content of the wrapped leaf reader.
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public static interface CoreClosedListener {
|
||||
/** Invoked when the shared core of the original {@code
|
||||
* SegmentReader} has closed. The provided {@code
|
||||
* ownerCoreCacheKey} will be the same key as the one
|
||||
* returned by {@link LeafReader#getCoreCacheKey()}. */
|
||||
void onClose(Object ownerCoreCacheKey) throws IOException;
|
||||
}
|
||||
|
||||
private static class CoreClosedListenerWrapper implements ReaderClosedListener {
|
||||
|
||||
private final CoreClosedListener listener;
|
||||
|
||||
CoreClosedListenerWrapper(CoreClosedListener listener) {
|
||||
this.listener = listener;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onClose(IndexReader reader) throws IOException {
|
||||
listener.onClose(reader.getCoreCacheKey());
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return listener.hashCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object other) {
|
||||
if (!(other instanceof CoreClosedListenerWrapper)) {
|
||||
return false;
|
||||
}
|
||||
return listener.equals(((CoreClosedListenerWrapper) other).listener);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/** Add a {@link CoreClosedListener} as a {@link ReaderClosedListener}. This
|
||||
* method is typically useful for {@link LeafReader} implementations that
|
||||
* don't have the concept of a core that is shared across several
|
||||
* {@link LeafReader} instances in which case the {@link CoreClosedListener}
|
||||
* is called when closing the reader. */
|
||||
protected static void addCoreClosedListenerAsReaderClosedListener(IndexReader reader, CoreClosedListener listener) {
|
||||
reader.addReaderClosedListener(new CoreClosedListenerWrapper(listener));
|
||||
}
|
||||
|
||||
/** Remove a {@link CoreClosedListener} which has been added with
|
||||
* {@link #addCoreClosedListenerAsReaderClosedListener(IndexReader, CoreClosedListener)}. */
|
||||
protected static void removeCoreClosedListenerAsReaderClosedListener(IndexReader reader, CoreClosedListener listener) {
|
||||
reader.removeReaderClosedListener(new CoreClosedListenerWrapper(listener));
|
||||
}
|
||||
|
||||
/** Expert: adds a CoreClosedListener to this reader's shared core
|
||||
* @lucene.experimental */
|
||||
public abstract void addCoreClosedListener(CoreClosedListener listener);
|
||||
|
||||
/** Expert: removes a CoreClosedListener from this reader's shared core
|
||||
* @lucene.experimental */
|
||||
public abstract void removeCoreClosedListener(CoreClosedListener listener);
|
||||
public abstract CacheHelper getCoreCacheHelper();
|
||||
|
||||
/**
|
||||
* Returns {@link Fields} for this reader.
|
||||
|
|
|
@ -70,16 +70,6 @@ class MergeReaderWrapper extends LeafReader {
|
|||
this.vectors = vectors;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addCoreClosedListener(CoreClosedListener listener) {
|
||||
in.addCoreClosedListener(listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeCoreClosedListener(CoreClosedListener listener) {
|
||||
in.removeCoreClosedListener(listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Fields fields() throws IOException {
|
||||
return fields;
|
||||
|
@ -224,13 +214,13 @@ class MergeReaderWrapper extends LeafReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Object getCoreCacheKey() {
|
||||
return in.getCoreCacheKey();
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return in.getCoreCacheHelper();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getCombinedCoreAndDeletesKey() {
|
||||
return in.getCombinedCoreAndDeletesKey();
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return in.getReaderCacheHelper();
|
||||
}
|
||||
|
||||
private void checkBounds(int docID) {
|
||||
|
|
|
@ -598,7 +598,9 @@ public class MultiDocValues {
|
|||
if (anyReal == false) {
|
||||
return null;
|
||||
} else {
|
||||
OrdinalMap mapping = OrdinalMap.build(r.getCoreCacheKey(), values, PackedInts.DEFAULT);
|
||||
IndexReader.CacheHelper cacheHelper = r.getReaderCacheHelper();
|
||||
IndexReader.CacheKey owner = cacheHelper == null ? null : cacheHelper.getKey();
|
||||
OrdinalMap mapping = OrdinalMap.build(owner, values, PackedInts.DEFAULT);
|
||||
return new MultiSortedDocValues(values, starts, mapping, totalCost);
|
||||
}
|
||||
}
|
||||
|
@ -640,7 +642,9 @@ public class MultiDocValues {
|
|||
if (anyReal == false) {
|
||||
return null;
|
||||
} else {
|
||||
OrdinalMap mapping = OrdinalMap.build(r.getCoreCacheKey(), values, PackedInts.DEFAULT);
|
||||
IndexReader.CacheHelper cacheHelper = r.getReaderCacheHelper();
|
||||
IndexReader.CacheKey owner = cacheHelper == null ? null : cacheHelper.getKey();
|
||||
OrdinalMap mapping = OrdinalMap.build(owner, values, PackedInts.DEFAULT);
|
||||
return new MultiSortedSetDocValues(values, starts, mapping, totalCost);
|
||||
}
|
||||
}
|
||||
|
@ -710,9 +714,9 @@ public class MultiDocValues {
|
|||
/**
|
||||
* Create an ordinal map that uses the number of unique values of each
|
||||
* {@link SortedDocValues} instance as a weight.
|
||||
* @see #build(Object, TermsEnum[], long[], float)
|
||||
* @see #build(IndexReader.CacheKey, TermsEnum[], long[], float)
|
||||
*/
|
||||
public static OrdinalMap build(Object owner, SortedDocValues[] values, float acceptableOverheadRatio) throws IOException {
|
||||
public static OrdinalMap build(IndexReader.CacheKey owner, SortedDocValues[] values, float acceptableOverheadRatio) throws IOException {
|
||||
final TermsEnum[] subs = new TermsEnum[values.length];
|
||||
final long[] weights = new long[values.length];
|
||||
for (int i = 0; i < values.length; ++i) {
|
||||
|
@ -725,9 +729,9 @@ public class MultiDocValues {
|
|||
/**
|
||||
* Create an ordinal map that uses the number of unique values of each
|
||||
* {@link SortedSetDocValues} instance as a weight.
|
||||
* @see #build(Object, TermsEnum[], long[], float)
|
||||
* @see #build(IndexReader.CacheKey, TermsEnum[], long[], float)
|
||||
*/
|
||||
public static OrdinalMap build(Object owner, SortedSetDocValues[] values, float acceptableOverheadRatio) throws IOException {
|
||||
public static OrdinalMap build(IndexReader.CacheKey owner, SortedSetDocValues[] values, float acceptableOverheadRatio) throws IOException {
|
||||
final TermsEnum[] subs = new TermsEnum[values.length];
|
||||
final long[] weights = new long[values.length];
|
||||
for (int i = 0; i < values.length; ++i) {
|
||||
|
@ -748,7 +752,7 @@ public class MultiDocValues {
|
|||
* to the other subs
|
||||
* @throws IOException if an I/O error occurred.
|
||||
*/
|
||||
public static OrdinalMap build(Object owner, TermsEnum subs[], long[] weights, float acceptableOverheadRatio) throws IOException {
|
||||
public static OrdinalMap build(IndexReader.CacheKey owner, TermsEnum subs[], long[] weights, float acceptableOverheadRatio) throws IOException {
|
||||
if (subs.length != weights.length) {
|
||||
throw new IllegalArgumentException("subs and weights must have the same length");
|
||||
}
|
||||
|
@ -761,7 +765,7 @@ public class MultiDocValues {
|
|||
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(OrdinalMap.class);
|
||||
|
||||
/** Cache key of whoever asked for this awful thing */
|
||||
public final Object owner;
|
||||
public final IndexReader.CacheKey owner;
|
||||
// globalOrd -> (globalOrd - segmentOrd) where segmentOrd is the the ordinal in the first segment that contains this term
|
||||
final PackedLongValues globalOrdDeltas;
|
||||
// globalOrd -> first segment container
|
||||
|
@ -773,7 +777,7 @@ public class MultiDocValues {
|
|||
// ram usage
|
||||
final long ramBytesUsed;
|
||||
|
||||
OrdinalMap(Object owner, TermsEnum subs[], SegmentMap segmentMap, float acceptableOverheadRatio) throws IOException {
|
||||
OrdinalMap(IndexReader.CacheKey owner, TermsEnum subs[], SegmentMap segmentMap, float acceptableOverheadRatio) throws IOException {
|
||||
// create the ordinal mappings by pulling a termsenum over each sub's
|
||||
// unique terms, and walking a multitermsenum over those
|
||||
this.owner = owner;
|
||||
|
|
|
@ -65,6 +65,17 @@ public class MultiReader extends BaseCompositeReader<IndexReader> {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
// MultiReader instances can be short-lived, which would make caching trappy
|
||||
// so we do not cache on them, unless they wrap a single reader in which
|
||||
// case we delegate
|
||||
if (getSequentialSubReaders().size() == 1) {
|
||||
return getSequentialSubReaders().get(0).getReaderCacheHelper();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected synchronized void doClose() throws IOException {
|
||||
IOException ioe = null;
|
||||
|
|
|
@ -51,6 +51,7 @@ public class ParallelCompositeReader extends BaseCompositeReader<LeafReader> {
|
|||
private final boolean closeSubReaders;
|
||||
private final Set<IndexReader> completeReaderSet =
|
||||
Collections.newSetFromMap(new IdentityHashMap<IndexReader,Boolean>());
|
||||
private final CacheHelper cacheHelper;
|
||||
|
||||
/** Create a ParallelCompositeReader based on the provided
|
||||
* readers; auto-closes the given readers on {@link #close()}. */
|
||||
|
@ -80,6 +81,14 @@ public class ParallelCompositeReader extends BaseCompositeReader<LeafReader> {
|
|||
}
|
||||
// finally add our own synthetic readers, so we close or decRef them, too (it does not matter what we do)
|
||||
completeReaderSet.addAll(getSequentialSubReaders());
|
||||
// ParallelReader instances can be short-lived, which would make caching trappy
|
||||
// so we do not cache on them, unless they wrap a single reader in which
|
||||
// case we delegate
|
||||
if (readers.length == 1 && storedFieldReaders.length == 1 && readers[0] == storedFieldReaders[0]) {
|
||||
cacheHelper = readers[0].getReaderCacheHelper();
|
||||
} else {
|
||||
cacheHelper = null;
|
||||
}
|
||||
}
|
||||
|
||||
private static LeafReader[] prepareLeafReaders(CompositeReader[] readers, CompositeReader[] storedFieldsReaders) throws IOException {
|
||||
|
@ -143,6 +152,11 @@ public class ParallelCompositeReader extends BaseCompositeReader<LeafReader> {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return cacheHelper;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected synchronized void doClose() throws IOException {
|
||||
IOException ioe = null;
|
||||
|
|
|
@ -159,16 +159,6 @@ public class ParallelLeafReader extends LeafReader {
|
|||
return buffer.append(')').toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addCoreClosedListener(CoreClosedListener listener) {
|
||||
addCoreClosedListenerAsReaderClosedListener(this, listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeCoreClosedListener(CoreClosedListener listener) {
|
||||
removeCoreClosedListenerAsReaderClosedListener(this, listener);
|
||||
}
|
||||
|
||||
// Single instance of this, per ParallelReader instance
|
||||
private final class ParallelFields extends Fields {
|
||||
final Map<String,Terms> fields = new TreeMap<>();
|
||||
|
@ -241,6 +231,32 @@ public class ParallelLeafReader extends LeafReader {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
// ParallelReader instances can be short-lived, which would make caching trappy
|
||||
// so we do not cache on them, unless they wrap a single reader in which
|
||||
// case we delegate
|
||||
if (parallelReaders.length == 1
|
||||
&& storedFieldsReaders.length == 1
|
||||
&& parallelReaders[0] == storedFieldsReaders[0]) {
|
||||
return parallelReaders[0].getCoreCacheHelper();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
// ParallelReader instances can be short-lived, which would make caching trappy
|
||||
// so we do not cache on them, unless they wrap a single reader in which
|
||||
// case we delegate
|
||||
if (parallelReaders.length == 1
|
||||
&& storedFieldsReaders.length == 1
|
||||
&& parallelReaders[0] == storedFieldsReaders[0]) {
|
||||
return parallelReaders[0].getReaderCacheHelper();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Fields getTermVectors(int docID) throws IOException {
|
||||
ensureOpen();
|
||||
|
|
|
@ -33,7 +33,8 @@ import org.apache.lucene.codecs.PointsReader;
|
|||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsReader;
|
||||
import org.apache.lucene.codecs.TermVectorsReader;
|
||||
import org.apache.lucene.index.LeafReader.CoreClosedListener;
|
||||
import org.apache.lucene.index.IndexReader.CacheKey;
|
||||
import org.apache.lucene.index.IndexReader.ClosedListener;
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
|
@ -84,8 +85,8 @@ final class SegmentCoreReaders {
|
|||
}
|
||||
};
|
||||
|
||||
private final Set<CoreClosedListener> coreClosedListeners =
|
||||
Collections.synchronizedSet(new LinkedHashSet<CoreClosedListener>());
|
||||
private final Set<IndexReader.ClosedListener> coreClosedListeners =
|
||||
Collections.synchronizedSet(new LinkedHashSet<IndexReader.ClosedListener>());
|
||||
|
||||
SegmentCoreReaders(Directory dir, SegmentCommitInfo si, IOContext context) throws IOException {
|
||||
|
||||
|
@ -176,13 +177,31 @@ final class SegmentCoreReaders {
|
|||
}
|
||||
}
|
||||
|
||||
private final IndexReader.CacheHelper cacheHelper = new IndexReader.CacheHelper() {
|
||||
private final IndexReader.CacheKey cacheKey = new IndexReader.CacheKey();
|
||||
|
||||
@Override
|
||||
public CacheKey getKey() {
|
||||
return cacheKey;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addClosedListener(ClosedListener listener) {
|
||||
coreClosedListeners.add(listener);
|
||||
}
|
||||
};
|
||||
|
||||
IndexReader.CacheHelper getCacheHelper() {
|
||||
return cacheHelper;
|
||||
}
|
||||
|
||||
private void notifyCoreClosedListeners(Throwable th) throws IOException {
|
||||
synchronized(coreClosedListeners) {
|
||||
for (CoreClosedListener listener : coreClosedListeners) {
|
||||
for (IndexReader.ClosedListener listener : coreClosedListeners) {
|
||||
// SegmentReader uses our instance as its
|
||||
// coreCacheKey:
|
||||
try {
|
||||
listener.onClose(this);
|
||||
listener.onClose(cacheHelper.getKey());
|
||||
} catch (Throwable t) {
|
||||
if (th == null) {
|
||||
th = t;
|
||||
|
@ -195,14 +214,6 @@ final class SegmentCoreReaders {
|
|||
}
|
||||
}
|
||||
|
||||
void addCoreClosedListener(CoreClosedListener listener) {
|
||||
coreClosedListeners.add(listener);
|
||||
}
|
||||
|
||||
void removeCoreClosedListener(CoreClosedListener listener) {
|
||||
coreClosedListeners.remove(listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "SegmentCoreReader(" + segment + ")";
|
||||
|
|
|
@ -19,6 +19,8 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.CopyOnWriteArraySet;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
|
@ -32,6 +34,7 @@ import org.apache.lucene.search.Sort;
|
|||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
/**
|
||||
* IndexReader implementation over a single segment.
|
||||
|
@ -282,32 +285,48 @@ public final class SegmentReader extends CodecReader {
|
|||
return si.info.dir;
|
||||
}
|
||||
|
||||
// This is necessary so that cloned SegmentReaders (which
|
||||
// share the underlying postings data) will map to the
|
||||
// same entry for CachingWrapperFilter. See LUCENE-1579.
|
||||
private final Set<ClosedListener> readerClosedListeners = new CopyOnWriteArraySet<>();
|
||||
|
||||
@Override
|
||||
public Object getCoreCacheKey() {
|
||||
// NOTE: if this ever changes, be sure to fix
|
||||
// SegmentCoreReader.notifyCoreClosedListeners to match!
|
||||
// Today it passes "this" as its coreCacheKey:
|
||||
return core;
|
||||
void notifyReaderClosedListeners(Throwable th) throws IOException {
|
||||
synchronized(readerClosedListeners) {
|
||||
for(ClosedListener listener : readerClosedListeners) {
|
||||
try {
|
||||
listener.onClose(cacheHelper.getKey());
|
||||
} catch (Throwable t) {
|
||||
if (th == null) {
|
||||
th = t;
|
||||
} else {
|
||||
th.addSuppressed(t);
|
||||
}
|
||||
}
|
||||
}
|
||||
IOUtils.reThrow(th);
|
||||
}
|
||||
}
|
||||
|
||||
private final IndexReader.CacheHelper cacheHelper = new IndexReader.CacheHelper() {
|
||||
private final IndexReader.CacheKey cacheKey = new IndexReader.CacheKey();
|
||||
|
||||
@Override
|
||||
public CacheKey getKey() {
|
||||
return cacheKey;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getCombinedCoreAndDeletesKey() {
|
||||
return this;
|
||||
public void addClosedListener(ClosedListener listener) {
|
||||
readerClosedListeners.add(listener);
|
||||
}
|
||||
};
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return cacheHelper;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addCoreClosedListener(CoreClosedListener listener) {
|
||||
ensureOpen();
|
||||
core.addCoreClosedListener(listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeCoreClosedListener(CoreClosedListener listener) {
|
||||
ensureOpen();
|
||||
core.removeCoreClosedListener(listener);
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return core.getCacheHelper();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -113,13 +113,13 @@ public final class SlowCodecReaderWrapper {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void addCoreClosedListener(CoreClosedListener listener) {
|
||||
reader.addCoreClosedListener(listener);
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return reader.getCoreCacheHelper();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeCoreClosedListener(CoreClosedListener listener) {
|
||||
reader.removeCoreClosedListener(listener);
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return reader.getReaderCacheHelper();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -1246,4 +1246,16 @@ class SortingLeafReader extends FilterLeafReader {
|
|||
public String toString() {
|
||||
return "SortingLeafReader(" + in + ")";
|
||||
}
|
||||
|
||||
// no caching on sorted views
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,6 +25,8 @@ import java.util.Collections;
|
|||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.CopyOnWriteArraySet;
|
||||
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
@ -469,4 +471,44 @@ public final class StandardDirectoryReader extends DirectoryReader {
|
|||
return reader;
|
||||
}
|
||||
}
|
||||
|
||||
private final Set<ClosedListener> readerClosedListeners = new CopyOnWriteArraySet<>();
|
||||
|
||||
private final CacheHelper cacheHelper = new CacheHelper() {
|
||||
private final CacheKey cacheKey = new CacheKey();
|
||||
|
||||
@Override
|
||||
public CacheKey getKey() {
|
||||
return cacheKey;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addClosedListener(ClosedListener listener) {
|
||||
readerClosedListeners.add(listener);
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
@Override
|
||||
void notifyReaderClosedListeners(Throwable th) throws IOException {
|
||||
synchronized(readerClosedListeners) {
|
||||
for(ClosedListener listener : readerClosedListeners) {
|
||||
try {
|
||||
listener.onClose(cacheHelper.getKey());
|
||||
} catch (Throwable t) {
|
||||
if (th == null) {
|
||||
th = t;
|
||||
} else {
|
||||
th.addSuppressed(t);
|
||||
}
|
||||
}
|
||||
}
|
||||
IOUtils.reThrow(th);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return cacheHelper;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -32,6 +32,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
|||
import java.util.concurrent.locks.ReentrantLock;
|
||||
import java.util.function.Predicate;
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.ReaderUtil;
|
||||
|
@ -107,7 +108,7 @@ public class LRUQueryCache implements QueryCache, Accountable {
|
|||
// are only allowed to store sub-sets of the queries that are contained in
|
||||
// mostRecentlyUsedQueries. This is why write operations are performed under a lock
|
||||
private final Set<Query> mostRecentlyUsedQueries;
|
||||
private final Map<Object, LeafCache> cache;
|
||||
private final Map<IndexReader.CacheKey, LeafCache> cache;
|
||||
private final ReentrantLock lock;
|
||||
|
||||
// these variables are volatile so that we do not need to sync reads
|
||||
|
@ -264,11 +265,11 @@ public class LRUQueryCache implements QueryCache, Accountable {
|
|||
}
|
||||
}
|
||||
|
||||
DocIdSet get(Query key, LeafReaderContext context) {
|
||||
DocIdSet get(Query key, LeafReaderContext context, IndexReader.CacheHelper cacheHelper) {
|
||||
assert lock.isHeldByCurrentThread();
|
||||
assert key instanceof BoostQuery == false;
|
||||
assert key instanceof ConstantScoreQuery == false;
|
||||
final Object readerKey = context.reader().getCoreCacheKey();
|
||||
final IndexReader.CacheKey readerKey = cacheHelper.getKey();
|
||||
final LeafCache leafCache = cache.get(readerKey);
|
||||
if (leafCache == null) {
|
||||
onMiss(readerKey, key);
|
||||
|
@ -289,7 +290,7 @@ public class LRUQueryCache implements QueryCache, Accountable {
|
|||
return cached;
|
||||
}
|
||||
|
||||
void putIfAbsent(Query query, LeafReaderContext context, DocIdSet set) {
|
||||
void putIfAbsent(Query query, LeafReaderContext context, DocIdSet set, IndexReader.CacheHelper cacheHelper) {
|
||||
assert query instanceof BoostQuery == false;
|
||||
assert query instanceof ConstantScoreQuery == false;
|
||||
// under a lock to make sure that mostRecentlyUsedQueries and cache remain sync'ed
|
||||
|
@ -301,15 +302,15 @@ public class LRUQueryCache implements QueryCache, Accountable {
|
|||
} else {
|
||||
query = singleton;
|
||||
}
|
||||
final Object key = context.reader().getCoreCacheKey();
|
||||
final IndexReader.CacheKey key = cacheHelper.getKey();
|
||||
LeafCache leafCache = cache.get(key);
|
||||
if (leafCache == null) {
|
||||
leafCache = new LeafCache(key);
|
||||
final LeafCache previous = cache.put(context.reader().getCoreCacheKey(), leafCache);
|
||||
final LeafCache previous = cache.put(key, leafCache);
|
||||
ramBytesUsed += HASHTABLE_RAM_BYTES_PER_ENTRY;
|
||||
assert previous == null;
|
||||
// we just created a new leaf cache, need to register a close listener
|
||||
context.reader().addCoreClosedListener(this::clearCoreCacheKey);
|
||||
cacheHelper.addClosedListener(this::clearCoreCacheKey);
|
||||
}
|
||||
leafCache.putIfAbsent(query, set);
|
||||
evictIfNecessary();
|
||||
|
@ -720,6 +721,14 @@ public class LRUQueryCache implements QueryCache, Accountable {
|
|||
if (used.compareAndSet(false, true)) {
|
||||
policy.onUse(getQuery());
|
||||
}
|
||||
|
||||
// TODO: should it be pluggable, eg. for queries that run on doc values?
|
||||
final IndexReader.CacheHelper cacheHelper = context.reader().getCoreCacheHelper();
|
||||
if (cacheHelper == null) {
|
||||
// this segment is not suitable for caching
|
||||
return in.scorer(context);
|
||||
}
|
||||
|
||||
// Short-circuit: Check whether this segment is eligible for caching
|
||||
// before we take a lock because of #get
|
||||
if (shouldCache(context) == false) {
|
||||
|
@ -733,7 +742,7 @@ public class LRUQueryCache implements QueryCache, Accountable {
|
|||
|
||||
DocIdSet docIdSet;
|
||||
try {
|
||||
docIdSet = get(in.getQuery(), context);
|
||||
docIdSet = get(in.getQuery(), context, cacheHelper);
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
|
@ -741,7 +750,7 @@ public class LRUQueryCache implements QueryCache, Accountable {
|
|||
if (docIdSet == null) {
|
||||
if (policy.shouldCache(in.getQuery())) {
|
||||
docIdSet = cache(context);
|
||||
putIfAbsent(in.getQuery(), context, docIdSet);
|
||||
putIfAbsent(in.getQuery(), context, docIdSet, cacheHelper);
|
||||
} else {
|
||||
return in.scorer(context);
|
||||
}
|
||||
|
@ -764,6 +773,14 @@ public class LRUQueryCache implements QueryCache, Accountable {
|
|||
if (used.compareAndSet(false, true)) {
|
||||
policy.onUse(getQuery());
|
||||
}
|
||||
|
||||
// TODO: should it be pluggable, eg. for queries that run on doc values?
|
||||
final IndexReader.CacheHelper cacheHelper = context.reader().getCoreCacheHelper();
|
||||
if (cacheHelper == null) {
|
||||
// this segment is not suitable for caching
|
||||
return in.bulkScorer(context);
|
||||
}
|
||||
|
||||
// Short-circuit: Check whether this segment is eligible for caching
|
||||
// before we take a lock because of #get
|
||||
if (shouldCache(context) == false) {
|
||||
|
@ -777,7 +794,7 @@ public class LRUQueryCache implements QueryCache, Accountable {
|
|||
|
||||
DocIdSet docIdSet;
|
||||
try {
|
||||
docIdSet = get(in.getQuery(), context);
|
||||
docIdSet = get(in.getQuery(), context, cacheHelper);
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
|
@ -785,7 +802,7 @@ public class LRUQueryCache implements QueryCache, Accountable {
|
|||
if (docIdSet == null) {
|
||||
if (policy.shouldCache(in.getQuery())) {
|
||||
docIdSet = cache(context);
|
||||
putIfAbsent(in.getQuery(), context, docIdSet);
|
||||
putIfAbsent(in.getQuery(), context, docIdSet, cacheHelper);
|
||||
} else {
|
||||
return in.bulkScorer(context);
|
||||
}
|
||||
|
|
|
@ -239,6 +239,11 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
|
|||
// throw the first exception
|
||||
IOUtils.reThrow(firstExc);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -324,7 +329,7 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
private class ParallelReaderClosed implements LeafReader.ReaderClosedListener {
|
||||
private class ParallelReaderClosed implements IndexReader.ClosedListener {
|
||||
private final SegmentIDAndGen segIDGen;
|
||||
private final Directory dir;
|
||||
|
||||
|
@ -334,7 +339,7 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void onClose(IndexReader ignored) {
|
||||
public void onClose(IndexReader.CacheKey ignored) {
|
||||
try {
|
||||
// TODO: make this sync finer, i.e. just the segment + schemaGen
|
||||
synchronized(ReindexingReader.this) {
|
||||
|
@ -421,7 +426,7 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
|
|||
// the pruning may remove our directory:
|
||||
closedSegments.remove(segIDGen);
|
||||
|
||||
parLeafReader.addReaderClosedListener(new ParallelReaderClosed(segIDGen, dir));
|
||||
parLeafReader.getReaderCacheHelper().addClosedListener(new ParallelReaderClosed(segIDGen, dir));
|
||||
|
||||
} else {
|
||||
// Used only for merged segment warming:
|
||||
|
|
|
@ -927,14 +927,14 @@ public class TestDirectoryReader extends LuceneTestCase {
|
|||
writer.commit();
|
||||
final DirectoryReader reader = writer.getReader();
|
||||
final int[] closeCount = new int[1];
|
||||
final IndexReader.ReaderClosedListener listener = new IndexReader.ReaderClosedListener() {
|
||||
final IndexReader.ClosedListener listener = new IndexReader.ClosedListener() {
|
||||
@Override
|
||||
public void onClose(IndexReader reader) {
|
||||
public void onClose(IndexReader.CacheKey key) {
|
||||
closeCount[0]++;
|
||||
}
|
||||
};
|
||||
|
||||
reader.addReaderClosedListener(listener);
|
||||
reader.getReaderCacheHelper().addClosedListener(listener);
|
||||
|
||||
reader.close();
|
||||
|
||||
|
@ -943,7 +943,7 @@ public class TestDirectoryReader extends LuceneTestCase {
|
|||
writer.close();
|
||||
|
||||
DirectoryReader reader2 = DirectoryReader.open(dir);
|
||||
reader2.addReaderClosedListener(listener);
|
||||
reader2.getReaderCacheHelper().addClosedListener(listener);
|
||||
|
||||
closeCount[0] = 0;
|
||||
reader2.close();
|
||||
|
|
|
@ -811,7 +811,8 @@ public class TestDirectoryReaderReopen extends LuceneTestCase {
|
|||
assertEquals(1, oldest.leaves().size());
|
||||
|
||||
// sharing same core
|
||||
assertSame(latest.leaves().get(0).reader().getCoreCacheKey(), oldest.leaves().get(0).reader().getCoreCacheKey());
|
||||
assertSame(latest.leaves().get(0).reader().getCoreCacheHelper().getKey(),
|
||||
oldest.leaves().get(0).reader().getCoreCacheHelper().getKey());
|
||||
|
||||
latest.close();
|
||||
oldest.close();
|
||||
|
@ -861,7 +862,8 @@ public class TestDirectoryReaderReopen extends LuceneTestCase {
|
|||
assertEquals(1, oldest.leaves().size());
|
||||
|
||||
// sharing same core
|
||||
assertSame(latest.leaves().get(0).reader().getCoreCacheKey(), oldest.leaves().get(0).reader().getCoreCacheKey());
|
||||
assertSame(latest.leaves().get(0).reader().getCoreCacheHelper().getKey(),
|
||||
oldest.leaves().get(0).reader().getCoreCacheHelper().getKey());
|
||||
|
||||
latest.close();
|
||||
oldest.close();
|
||||
|
@ -901,7 +903,8 @@ public class TestDirectoryReaderReopen extends LuceneTestCase {
|
|||
assertEquals(1, oldest.leaves().size());
|
||||
|
||||
// sharing same core
|
||||
assertSame(latest.leaves().get(0).reader().getCoreCacheKey(), oldest.leaves().get(0).reader().getCoreCacheKey());
|
||||
assertSame(latest.leaves().get(0).reader().getCoreCacheHelper().getKey(),
|
||||
oldest.leaves().get(0).reader().getCoreCacheHelper().getKey());
|
||||
|
||||
NumericDocValues values = getOnlyLeafReader(oldest).getNumericDocValues("dv");
|
||||
assertEquals(0, values.nextDoc());
|
||||
|
@ -948,7 +951,8 @@ public class TestDirectoryReaderReopen extends LuceneTestCase {
|
|||
assertEquals(1, oldest.leaves().size());
|
||||
|
||||
// sharing same core
|
||||
assertSame(latest.leaves().get(0).reader().getCoreCacheKey(), oldest.leaves().get(0).reader().getCoreCacheKey());
|
||||
assertSame(latest.leaves().get(0).reader().getCoreCacheHelper().getKey(),
|
||||
oldest.leaves().get(0).reader().getCoreCacheHelper().getKey());
|
||||
|
||||
NumericDocValues values = getOnlyLeafReader(oldest).getNumericDocValues("dv");
|
||||
assertEquals(0, values.nextDoc());
|
||||
|
|
|
@ -86,6 +86,16 @@ public class TestExitableDirectoryReader extends LuceneTestCase {
|
|||
public Fields fields() throws IOException {
|
||||
return new TestFields(super.fields());
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return in.getCoreCacheHelper();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return in.getReaderCacheHelper();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -50,6 +50,11 @@ public class TestFilterDirectoryReader extends LuceneTestCase {
|
|||
return new DummyFilterDirectoryReader(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return in.getReaderCacheHelper();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public void testDoubleClose() throws IOException {
|
||||
|
|
|
@ -106,6 +106,16 @@ public class TestFilterLeafReader extends LuceneTestCase {
|
|||
public Fields fields() throws IOException {
|
||||
return new TestFields(super.fields());
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -196,7 +206,16 @@ public class TestFilterLeafReader extends LuceneTestCase {
|
|||
w.addDocument(new Document());
|
||||
DirectoryReader dr = w.getReader();
|
||||
LeafReader r = dr.leaves().get(0).reader();
|
||||
FilterLeafReader r2 = new FilterLeafReader(r) {};
|
||||
FilterLeafReader r2 = new FilterLeafReader(r) {
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return in.getCoreCacheHelper();
|
||||
}
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return in.getReaderCacheHelper();
|
||||
}
|
||||
};
|
||||
assertEquals(r, r2.getDelegate());
|
||||
assertEquals(r, FilterLeafReader.unwrap(r2));
|
||||
w.close();
|
||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
|
@ -46,13 +45,24 @@ public class TestIndexReaderClose extends LuceneTestCase {
|
|||
final boolean throwOnClose = !rarely();
|
||||
LeafReader leaf = getOnlyLeafReader(open);
|
||||
FilterLeafReader reader = new FilterLeafReader(leaf) {
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return in.getCoreCacheHelper();
|
||||
}
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return in.getReaderCacheHelper();
|
||||
}
|
||||
@Override
|
||||
protected void doClose() throws IOException {
|
||||
try {
|
||||
super.doClose();
|
||||
} finally {
|
||||
if (throwOnClose) {
|
||||
throw new IllegalStateException("BOOM!");
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
int listenerCount = random().nextInt(20);
|
||||
AtomicInteger count = new AtomicInteger();
|
||||
|
@ -60,14 +70,14 @@ public class TestIndexReaderClose extends LuceneTestCase {
|
|||
for (int i = 0; i < listenerCount; i++) {
|
||||
if (rarely()) {
|
||||
faultySet = true;
|
||||
reader.addReaderClosedListener(new FaultyListener());
|
||||
reader.getReaderCacheHelper().addClosedListener(new FaultyListener());
|
||||
} else {
|
||||
count.incrementAndGet();
|
||||
reader.addReaderClosedListener(new CountListener(count));
|
||||
reader.getReaderCacheHelper().addClosedListener(new CountListener(count));
|
||||
}
|
||||
}
|
||||
if (!faultySet && !throwOnClose) {
|
||||
reader.addReaderClosedListener(new FaultyListener());
|
||||
reader.getReaderCacheHelper().addClosedListener(new FaultyListener());
|
||||
}
|
||||
|
||||
IllegalStateException expected = expectThrows(IllegalStateException.class, () -> {
|
||||
|
@ -106,31 +116,19 @@ public class TestIndexReaderClose extends LuceneTestCase {
|
|||
w.close();
|
||||
|
||||
final IndexReader reader = DirectoryReader.open(w.w.getDirectory());
|
||||
// We explicitly define a different cache key
|
||||
final Object coreCacheKey = new Object();
|
||||
final LeafReader leafReader = new FilterLeafReader(getOnlyLeafReader(reader)) {
|
||||
@Override
|
||||
public Object getCoreCacheKey() {
|
||||
return coreCacheKey;
|
||||
}
|
||||
};
|
||||
final LeafReader leafReader = new AssertingLeafReader(getOnlyLeafReader(reader));
|
||||
|
||||
final int numListeners = TestUtil.nextInt(random(), 1, 10);
|
||||
final List<LeafReader.CoreClosedListener> listeners = new ArrayList<>();
|
||||
final List<IndexReader.ClosedListener> listeners = new ArrayList<>();
|
||||
AtomicInteger counter = new AtomicInteger(numListeners);
|
||||
|
||||
for (int i = 0; i < numListeners; ++i) {
|
||||
CountCoreListener listener = new CountCoreListener(counter, coreCacheKey);
|
||||
CountCoreListener listener = new CountCoreListener(counter, leafReader.getCoreCacheHelper().getKey());
|
||||
listeners.add(listener);
|
||||
leafReader.addCoreClosedListener(listener);
|
||||
leafReader.getCoreCacheHelper().addClosedListener(listener);
|
||||
}
|
||||
for (int i = 0; i < 100; ++i) {
|
||||
leafReader.addCoreClosedListener(listeners.get(random().nextInt(listeners.size())));
|
||||
}
|
||||
final int removed = random().nextInt(numListeners);
|
||||
Collections.shuffle(listeners, random());
|
||||
for (int i = 0; i < removed; ++i) {
|
||||
leafReader.removeCoreClosedListener(listeners.get(i));
|
||||
leafReader.getCoreCacheHelper().addClosedListener(listeners.get(random().nextInt(listeners.size())));
|
||||
}
|
||||
assertEquals(numListeners, counter.get());
|
||||
// make sure listeners are registered on the wrapped reader and that closing any of them has the same effect
|
||||
|
@ -139,11 +137,11 @@ public class TestIndexReaderClose extends LuceneTestCase {
|
|||
} else {
|
||||
leafReader.close();
|
||||
}
|
||||
assertEquals(removed, counter.get());
|
||||
assertEquals(0, counter.get());
|
||||
w.w.getDirectory().close();
|
||||
}
|
||||
|
||||
private static final class CountCoreListener implements LeafReader.CoreClosedListener {
|
||||
private static final class CountCoreListener implements IndexReader.ClosedListener {
|
||||
|
||||
private final AtomicInteger count;
|
||||
private final Object coreCacheKey;
|
||||
|
@ -154,14 +152,14 @@ public class TestIndexReaderClose extends LuceneTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void onClose(Object coreCacheKey) {
|
||||
public void onClose(IndexReader.CacheKey coreCacheKey) {
|
||||
assertSame(this.coreCacheKey, coreCacheKey);
|
||||
count.decrementAndGet();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private static final class CountListener implements IndexReader.ReaderClosedListener {
|
||||
private static final class CountListener implements IndexReader.ClosedListener {
|
||||
private final AtomicInteger count;
|
||||
|
||||
public CountListener(AtomicInteger count) {
|
||||
|
@ -169,15 +167,15 @@ public class TestIndexReaderClose extends LuceneTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void onClose(IndexReader reader) {
|
||||
public void onClose(IndexReader.CacheKey cacheKey) {
|
||||
count.decrementAndGet();
|
||||
}
|
||||
}
|
||||
|
||||
private static final class FaultyListener implements IndexReader.ReaderClosedListener {
|
||||
private static final class FaultyListener implements IndexReader.ClosedListener {
|
||||
|
||||
@Override
|
||||
public void onClose(IndexReader reader) {
|
||||
public void onClose(IndexReader.CacheKey cacheKey) {
|
||||
throw new IllegalStateException("GRRRRRRRRRRRR!");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -265,5 +265,15 @@ public class TestMultiTermsEnum extends LuceneTestCase {
|
|||
delegate.close();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -126,7 +126,7 @@ public class TestParallelCompositeReader extends LuceneTestCase {
|
|||
dir2.close();
|
||||
}
|
||||
|
||||
private void testReaderClosedListener(boolean closeSubReaders, int wrapMultiReaderType) throws IOException {
|
||||
private void testReaderClosedListener1(boolean closeSubReaders, int wrapMultiReaderType) throws IOException {
|
||||
final Directory dir1 = getDir1(random());
|
||||
final CompositeReader ir2, ir1 = DirectoryReader.open(dir1);
|
||||
switch (wrapMultiReaderType) {
|
||||
|
@ -147,18 +147,19 @@ public class TestParallelCompositeReader extends LuceneTestCase {
|
|||
new CompositeReader[] {ir2},
|
||||
new CompositeReader[] {ir2});
|
||||
|
||||
final int[] listenerClosedCount = new int[1];
|
||||
|
||||
assertEquals(3, pr.leaves().size());
|
||||
assertEquals(ir1.getReaderCacheHelper(), pr.getReaderCacheHelper());
|
||||
|
||||
int i = 0;
|
||||
for(LeafReaderContext cxt : pr.leaves()) {
|
||||
cxt.reader().addReaderClosedListener(reader -> listenerClosedCount[0]++);
|
||||
LeafReader originalLeaf = ir1.leaves().get(i++).reader();
|
||||
assertEquals(originalLeaf.getCoreCacheHelper(), cxt.reader().getCoreCacheHelper());
|
||||
assertEquals(originalLeaf.getReaderCacheHelper(), cxt.reader().getReaderCacheHelper());
|
||||
}
|
||||
pr.close();
|
||||
if (!closeSubReaders) {
|
||||
ir1.close();
|
||||
}
|
||||
assertEquals(3, listenerClosedCount[0]);
|
||||
|
||||
// We have to close the extra MultiReader, because it will not close its own subreaders:
|
||||
if (wrapMultiReaderType == 2) {
|
||||
|
@ -168,23 +169,11 @@ public class TestParallelCompositeReader extends LuceneTestCase {
|
|||
}
|
||||
|
||||
public void testReaderClosedListener1() throws Exception {
|
||||
testReaderClosedListener(false, 0);
|
||||
}
|
||||
|
||||
public void testReaderClosedListener2() throws Exception {
|
||||
testReaderClosedListener(true, 0);
|
||||
}
|
||||
|
||||
public void testReaderClosedListener3() throws Exception {
|
||||
testReaderClosedListener(false, 1);
|
||||
}
|
||||
|
||||
public void testReaderClosedListener4() throws Exception {
|
||||
testReaderClosedListener(true, 1);
|
||||
}
|
||||
|
||||
public void testReaderClosedListener5() throws Exception {
|
||||
testReaderClosedListener(false, 2);
|
||||
testReaderClosedListener1(false, 0);
|
||||
testReaderClosedListener1(true, 0);
|
||||
testReaderClosedListener1(false, 1);
|
||||
testReaderClosedListener1(true, 1);
|
||||
testReaderClosedListener1(false, 2);
|
||||
}
|
||||
|
||||
public void testCloseInnerReader() throws Exception {
|
||||
|
|
|
@ -238,6 +238,16 @@ public class TermInSetQueryTest extends LuceneTestCase {
|
|||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -245,6 +255,11 @@ public class TermInSetQueryTest extends LuceneTestCase {
|
|||
return new TermsCountingDirectoryReaderWrapper(in, counter);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public void testPullOneTermsEnum() throws Exception {
|
||||
|
|
|
@ -42,8 +42,11 @@ import org.apache.lucene.document.Field.Store;
|
|||
import org.apache.lucene.document.StringField;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.FilterDirectoryReader;
|
||||
import org.apache.lucene.index.FilterLeafReader;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.NoMergePolicy;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
|
@ -607,12 +610,12 @@ public class TestLRUQueryCache extends LuceneTestCase {
|
|||
final int segmentCount2 = reader2.leaves().size();
|
||||
final IndexSearcher searcher2 = new IndexSearcher(reader2);
|
||||
|
||||
final Map<Object, Integer> indexId = new HashMap<>();
|
||||
final Map<IndexReader.CacheKey, Integer> indexId = new HashMap<>();
|
||||
for (LeafReaderContext ctx : reader1.leaves()) {
|
||||
indexId.put(ctx.reader().getCoreCacheKey(), 1);
|
||||
indexId.put(ctx.reader().getCoreCacheHelper().getKey(), 1);
|
||||
}
|
||||
for (LeafReaderContext ctx : reader2.leaves()) {
|
||||
indexId.put(ctx.reader().getCoreCacheKey(), 2);
|
||||
indexId.put(ctx.reader().getCoreCacheHelper().getKey(), 2);
|
||||
}
|
||||
|
||||
final AtomicLong hitCount1 = new AtomicLong();
|
||||
|
@ -1218,4 +1221,56 @@ public class TestLRUQueryCache extends LuceneTestCase {
|
|||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
// a reader whose sole purpose is to not be cacheable
|
||||
private static class DummyDirectoryReader extends FilterDirectoryReader {
|
||||
|
||||
public DummyDirectoryReader(DirectoryReader in) throws IOException {
|
||||
super(in, new SubReaderWrapper() {
|
||||
@Override
|
||||
public LeafReader wrap(LeafReader reader) {
|
||||
return new FilterLeafReader(reader) {
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
};
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
|
||||
return new DummyDirectoryReader(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
public void testReaderNotSuitedForCaching() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = newIndexWriterConfig().setMergePolicy(NoMergePolicy.INSTANCE);
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
|
||||
w.addDocument(new Document());
|
||||
DirectoryReader reader = new DummyDirectoryReader(w.getReader());
|
||||
IndexSearcher searcher = newSearcher(reader);
|
||||
searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE);
|
||||
|
||||
// don't cache if the reader does not expose a cache helper
|
||||
assertNull(reader.leaves().get(0).reader().getCoreCacheHelper());
|
||||
LRUQueryCache cache = new LRUQueryCache(2, 10000, context -> true);
|
||||
searcher.setQueryCache(cache);
|
||||
assertEquals(0, searcher.count(new DummyQuery()));
|
||||
assertEquals(0, cache.getCacheCount());
|
||||
reader.close();
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -443,6 +443,16 @@ public class TestSearcherManager extends ThreadedIndexingAndSearchingTestCase {
|
|||
public MyFilterLeafReader(LeafReader in) {
|
||||
super(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return in.getCoreCacheHelper();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return in.getReaderCacheHelper();
|
||||
}
|
||||
}
|
||||
|
||||
private static class MyFilterDirectoryReader extends FilterDirectoryReader {
|
||||
|
@ -462,6 +472,11 @@ public class TestSearcherManager extends ThreadedIndexingAndSearchingTestCase {
|
|||
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
|
||||
return new MyFilterDirectoryReader(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return in.getReaderCacheHelper();
|
||||
}
|
||||
}
|
||||
|
||||
// LUCENE-6087
|
||||
|
|
|
@ -109,6 +109,11 @@ public class TestTermQuery extends LuceneTestCase {
|
|||
return new NoSeekDirectoryReader(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return in.getReaderCacheHelper();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private static class NoSeekLeafReader extends FilterLeafReader {
|
||||
|
@ -149,6 +154,16 @@ public class TestTermQuery extends LuceneTestCase {
|
|||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return in.getCoreCacheHelper();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return in.getReaderCacheHelper();
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
}
|
||||
|
|
|
@ -179,6 +179,16 @@ public class TestTermScorer extends LuceneTestCase {
|
|||
// unreachable
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return in.getCoreCacheHelper();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return in.getReaderCacheHelper();
|
||||
}
|
||||
};
|
||||
// We don't use newSearcher because it sometimes runs checkIndex which loads norms
|
||||
IndexSearcher indexSearcher = new IndexSearcher(forbiddenNorms);
|
||||
|
|
|
@ -116,7 +116,8 @@ public class DefaultSortedSetDocValuesReaderState extends SortedSetDocValuesRead
|
|||
SortedSetDocValues dv = MultiDocValues.getSortedSetValues(origReader, field);
|
||||
if (dv instanceof MultiDocValues.MultiSortedSetDocValues) {
|
||||
map = ((MultiDocValues.MultiSortedSetDocValues)dv).mapping;
|
||||
if (map.owner == origReader.getCoreCacheKey()) {
|
||||
IndexReader.CacheHelper cacheHelper = origReader.getReaderCacheHelper();
|
||||
if (cacheHelper != null && map.owner == cacheHelper.getKey()) {
|
||||
cachedOrdMaps.put(field, map);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,6 +23,7 @@ import java.util.WeakHashMap;
|
|||
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.index.BinaryDocValues;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.Accountables;
|
||||
|
@ -67,7 +68,11 @@ public class CachedOrdinalsReader extends OrdinalsReader implements Accountable
|
|||
}
|
||||
|
||||
private synchronized CachedOrds getCachedOrds(LeafReaderContext context) throws IOException {
|
||||
Object cacheKey = context.reader().getCoreCacheKey();
|
||||
IndexReader.CacheHelper cacheHelper = context.reader().getCoreCacheHelper();
|
||||
if (cacheHelper == null) {
|
||||
throw new IllegalStateException("Cannot cache ordinals on leaf: " + context.reader());
|
||||
}
|
||||
Object cacheKey = cacheHelper.getKey();
|
||||
CachedOrds ords = ordsCache.get(cacheKey);
|
||||
if (ords == null) {
|
||||
ords = new CachedOrds(source.getReader(context), context.reader().maxDoc());
|
||||
|
|
|
@ -158,4 +158,14 @@ public class OrdinalMappingLeafReader extends FilterLeafReader {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -84,16 +84,6 @@ public class TermVectorLeafReader extends LeafReader {
|
|||
fieldInfos = new FieldInfos(new FieldInfo[]{fieldInfo});
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addCoreClosedListener(CoreClosedListener listener) {
|
||||
addCoreClosedListenerAsReaderClosedListener(this, listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeCoreClosedListener(CoreClosedListener listener) {
|
||||
removeCoreClosedListenerAsReaderClosedListener(this, listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doClose() throws IOException {
|
||||
}
|
||||
|
@ -178,4 +168,14 @@ public class TermVectorLeafReader extends LeafReader {
|
|||
public Sort getIndexSort() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -474,6 +474,16 @@ public class WeightedSpanTermExtractor {
|
|||
public NumericDocValues getNormValues(String field) throws IOException {
|
||||
return super.getNormValues(FIELD_NAME);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -584,6 +584,16 @@ public class PhraseHelper {
|
|||
public NumericDocValues getNormValues(String field) throws IOException {
|
||||
return super.getNormValues(fieldName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -127,4 +127,14 @@ final class TermVectorFilteredLeafReader extends FilterLeafReader {
|
|||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1047,6 +1047,11 @@ public class UnifiedHighlighter {
|
|||
protected void doClose() throws IOException {
|
||||
reader.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
@ -1066,6 +1071,16 @@ public class UnifiedHighlighter {
|
|||
return tvFields;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -132,6 +132,16 @@ public class TestUnifiedHighlighterTermVec extends LuceneTestCase {
|
|||
|
||||
return super.getTermVectors(docID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
|
@ -144,6 +154,11 @@ public class TestUnifiedHighlighterTermVec extends LuceneTestCase {
|
|||
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
|
||||
return new AssertOnceTermVecDirectoryReader(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
private static boolean calledBy(Class<?> clazz) {
|
||||
|
|
|
@ -21,6 +21,7 @@ import java.util.Collections;
|
|||
import java.util.Map;
|
||||
import java.util.WeakHashMap;
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexReaderContext;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
|
@ -39,7 +40,7 @@ import org.apache.lucene.util.BitSet;
|
|||
*/
|
||||
public class QueryBitSetProducer implements BitSetProducer {
|
||||
private final Query query;
|
||||
private final Map<Object,DocIdSet> cache = Collections.synchronizedMap(new WeakHashMap<>());
|
||||
final Map<IndexReader.CacheKey,DocIdSet> cache = Collections.synchronizedMap(new WeakHashMap<>());
|
||||
|
||||
/** Wraps another query's result and caches it into bitsets.
|
||||
* @param query Query to cache results of
|
||||
|
@ -59,9 +60,12 @@ public class QueryBitSetProducer implements BitSetProducer {
|
|||
@Override
|
||||
public BitSet getBitSet(LeafReaderContext context) throws IOException {
|
||||
final LeafReader reader = context.reader();
|
||||
final Object key = reader.getCoreCacheKey();
|
||||
final IndexReader.CacheHelper cacheHelper = reader.getCoreCacheHelper();
|
||||
|
||||
DocIdSet docIdSet = cache.get(key);
|
||||
DocIdSet docIdSet = null;
|
||||
if (cacheHelper != null) {
|
||||
docIdSet = cache.get(cacheHelper.getKey());
|
||||
}
|
||||
if (docIdSet == null) {
|
||||
final IndexReaderContext topLevelContext = ReaderUtil.getTopLevelContext(context);
|
||||
final IndexSearcher searcher = new IndexSearcher(topLevelContext);
|
||||
|
@ -74,7 +78,9 @@ public class QueryBitSetProducer implements BitSetProducer {
|
|||
} else {
|
||||
docIdSet = new BitDocIdSet(BitSet.of(s.iterator(), context.reader().maxDoc()));
|
||||
}
|
||||
cache.put(key, docIdSet);
|
||||
if (cacheHelper != null) {
|
||||
cache.put(cacheHelper.getKey(), docIdSet);
|
||||
}
|
||||
}
|
||||
return docIdSet == DocIdSet.EMPTY ? null : ((BitDocIdSet) docIdSet).bits();
|
||||
}
|
||||
|
|
|
@ -267,7 +267,7 @@ public class TestJoinUtil extends LuceneTestCase {
|
|||
values[i] = DocValues.getSorted(leafReader, joinField);
|
||||
}
|
||||
MultiDocValues.OrdinalMap ordinalMap = MultiDocValues.OrdinalMap.build(
|
||||
r.getCoreCacheKey(), values, PackedInts.DEFAULT
|
||||
null, values, PackedInts.DEFAULT
|
||||
);
|
||||
|
||||
Query toQuery = new TermQuery(new Term(typeField, "price"));
|
||||
|
@ -372,7 +372,7 @@ public class TestJoinUtil extends LuceneTestCase {
|
|||
values[i] = DocValues.getSorted(leafReader, joinField);
|
||||
}
|
||||
MultiDocValues.OrdinalMap ordinalMap = MultiDocValues.OrdinalMap.build(
|
||||
r.getCoreCacheKey(), values, PackedInts.DEFAULT
|
||||
null, values, PackedInts.DEFAULT
|
||||
);
|
||||
|
||||
Query toQuery = new TermQuery(new Term("price", "5.0"));
|
||||
|
@ -500,7 +500,7 @@ public class TestJoinUtil extends LuceneTestCase {
|
|||
values[leadContext.ord] = DocValues.getSorted(leadContext.reader(), "join_field");
|
||||
}
|
||||
MultiDocValues.OrdinalMap ordinalMap = MultiDocValues.OrdinalMap.build(
|
||||
searcher.getIndexReader().getCoreCacheKey(), values, PackedInts.DEFAULT
|
||||
null, values, PackedInts.DEFAULT
|
||||
);
|
||||
BooleanQuery.Builder fromQuery = new BooleanQuery.Builder();
|
||||
fromQuery.add(priceQuery, BooleanClause.Occur.MUST);
|
||||
|
@ -621,7 +621,7 @@ public class TestJoinUtil extends LuceneTestCase {
|
|||
values[leadContext.ord] = DocValues.getSorted(leadContext.reader(), "join_field");
|
||||
}
|
||||
MultiDocValues.OrdinalMap ordinalMap = MultiDocValues.OrdinalMap.build(
|
||||
searcher.getIndexReader().getCoreCacheKey(), values, PackedInts.DEFAULT
|
||||
null, values, PackedInts.DEFAULT
|
||||
);
|
||||
Query fromQuery = new TermQuery(new Term("type", "from"));
|
||||
Query toQuery = new TermQuery(new Term("type", "to"));
|
||||
|
@ -1336,7 +1336,7 @@ public class TestJoinUtil extends LuceneTestCase {
|
|||
values[leadContext.ord] = DocValues.getSorted(leadContext.reader(), "join_field");
|
||||
}
|
||||
context.ordinalMap = MultiDocValues.OrdinalMap.build(
|
||||
topLevelReader.getCoreCacheKey(), values, PackedInts.DEFAULT
|
||||
null, values, PackedInts.DEFAULT
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,110 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.search.join;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.FilterDirectoryReader;
|
||||
import org.apache.lucene.index.FilterLeafReader;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.NoMergePolicy;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||
import org.apache.lucene.search.MatchNoDocsQuery;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BitSet;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
public class TestQueryBitSetProducer extends LuceneTestCase {
|
||||
|
||||
public void testSimple() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = newIndexWriterConfig().setMergePolicy(NoMergePolicy.INSTANCE);
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
|
||||
w.addDocument(new Document());
|
||||
DirectoryReader reader = w.getReader();
|
||||
|
||||
QueryBitSetProducer producer = new QueryBitSetProducer(new MatchNoDocsQuery());
|
||||
assertNull(producer.getBitSet(reader.leaves().get(0)));
|
||||
assertEquals(1, producer.cache.size());
|
||||
|
||||
producer = new QueryBitSetProducer(new MatchAllDocsQuery());
|
||||
BitSet bitSet = producer.getBitSet(reader.leaves().get(0));
|
||||
assertEquals(1, bitSet.length());
|
||||
assertEquals(true, bitSet.get(0));
|
||||
assertEquals(1, producer.cache.size());
|
||||
|
||||
IOUtils.close(reader, w, dir);
|
||||
}
|
||||
|
||||
public void testReaderNotSuitedForCaching() throws IOException{
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = newIndexWriterConfig().setMergePolicy(NoMergePolicy.INSTANCE);
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
|
||||
w.addDocument(new Document());
|
||||
DirectoryReader reader = new DummyDirectoryReader(w.getReader());
|
||||
|
||||
QueryBitSetProducer producer = new QueryBitSetProducer(new MatchNoDocsQuery());
|
||||
assertNull(producer.getBitSet(reader.leaves().get(0)));
|
||||
assertEquals(0, producer.cache.size());
|
||||
|
||||
producer = new QueryBitSetProducer(new MatchAllDocsQuery());
|
||||
BitSet bitSet = producer.getBitSet(reader.leaves().get(0));
|
||||
assertEquals(1, bitSet.length());
|
||||
assertEquals(true, bitSet.get(0));
|
||||
assertEquals(0, producer.cache.size());
|
||||
|
||||
IOUtils.close(reader, w, dir);
|
||||
}
|
||||
|
||||
// a reader whose sole purpose is to not be cacheable
|
||||
private static class DummyDirectoryReader extends FilterDirectoryReader {
|
||||
|
||||
public DummyDirectoryReader(DirectoryReader in) throws IOException {
|
||||
super(in, new SubReaderWrapper() {
|
||||
@Override
|
||||
public LeafReader wrap(LeafReader reader) {
|
||||
return new FilterLeafReader(reader) {
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}};
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
|
||||
return new DummyDirectoryReader(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1182,16 +1182,6 @@ public class MemoryIndex {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addCoreClosedListener(CoreClosedListener listener) {
|
||||
addCoreClosedListenerAsReaderClosedListener(this, listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeCoreClosedListener(CoreClosedListener listener) {
|
||||
removeCoreClosedListenerAsReaderClosedListener(this, listener);
|
||||
}
|
||||
|
||||
private Info getInfoForExpectedDocValuesType(String fieldName, DocValuesType expectedType) {
|
||||
if (expectedType == DocValuesType.NONE) {
|
||||
return null;
|
||||
|
@ -1684,6 +1674,16 @@ public class MemoryIndex {
|
|||
public Sort getIndexSort() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -206,6 +206,11 @@ public class MultiPassIndexSplitter {
|
|||
@Override
|
||||
protected void doClose() {}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
|
||||
// no need to override numDocs/hasDeletions,
|
||||
// as we pass the subreaders directly to IW.addIndexes().
|
||||
}
|
||||
|
@ -247,5 +252,15 @@ public class MultiPassIndexSplitter {
|
|||
public Bits getLiveDocs() {
|
||||
return liveDocs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return in.getCoreCacheHelper();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -167,5 +167,15 @@ public class PKIndexSplitter {
|
|||
public Bits getLiveDocs() {
|
||||
return liveDocs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return in.getCoreCacheHelper();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -111,10 +111,14 @@ class SegmentInfosSearcherManager extends ReferenceManager<IndexSearcher> {
|
|||
}
|
||||
|
||||
private void addReaderClosedListener(IndexReader r) {
|
||||
IndexReader.CacheHelper cacheHelper = r.getReaderCacheHelper();
|
||||
if (cacheHelper == null) {
|
||||
throw new IllegalStateException("StandardDirectoryReader must support caching");
|
||||
}
|
||||
openReaderCount.incrementAndGet();
|
||||
r.addReaderClosedListener(new IndexReader.ReaderClosedListener() {
|
||||
cacheHelper.addClosedListener(new IndexReader.ClosedListener() {
|
||||
@Override
|
||||
public void onClose(IndexReader reader) {
|
||||
public void onClose(IndexReader.CacheKey cacheKey) {
|
||||
onReaderClosed();
|
||||
}
|
||||
});
|
||||
|
|
|
@ -39,4 +39,14 @@ public class AllDeletedFilterReader extends FilterLeafReader {
|
|||
public int numDocs() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return in.getCoreCacheHelper();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -41,13 +41,8 @@ public class AssertingDirectoryReader extends FilterDirectoryReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Object getCoreCacheKey() {
|
||||
return in.getCoreCacheKey();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getCombinedCoreAndDeletesKey() {
|
||||
return in.getCombinedCoreAndDeletesKey();
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return in.getReaderCacheHelper();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -51,13 +51,25 @@ public class AssertingLeafReader extends FilterLeafReader {
|
|||
assert in.numDeletedDocs() + in.numDocs() == in.maxDoc();
|
||||
assert !in.hasDeletions() || in.numDeletedDocs() > 0 && in.numDocs() < in.maxDoc();
|
||||
|
||||
addCoreClosedListener(ownerCoreCacheKey -> {
|
||||
final Object expectedKey = getCoreCacheKey();
|
||||
assert expectedKey == ownerCoreCacheKey
|
||||
: "Core closed listener called on a different key " + expectedKey + " <> " + ownerCoreCacheKey;
|
||||
CacheHelper coreCacheHelper = in.getCoreCacheHelper();
|
||||
if (coreCacheHelper != null) {
|
||||
coreCacheHelper.addClosedListener(cacheKey -> {
|
||||
final Object expectedKey = coreCacheHelper.getKey();
|
||||
assert expectedKey == cacheKey
|
||||
: "Core closed listener called on a different key " + expectedKey + " <> " + cacheKey;
|
||||
});
|
||||
}
|
||||
|
||||
CacheHelper readerCacheHelper = in.getReaderCacheHelper();
|
||||
if (readerCacheHelper != null) {
|
||||
readerCacheHelper.addClosedListener(cacheKey -> {
|
||||
final Object expectedKey = readerCacheHelper.getKey();
|
||||
assert expectedKey == cacheKey
|
||||
: "Core closed listener called on a different key " + expectedKey + " <> " + cacheKey;
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Fields fields() throws IOException {
|
||||
return new AssertingFields(super.fields());
|
||||
|
@ -1137,12 +1149,12 @@ public class AssertingLeafReader extends FilterLeafReader {
|
|||
// we don't change behavior of the reader: just validate the API.
|
||||
|
||||
@Override
|
||||
public Object getCoreCacheKey() {
|
||||
return in.getCoreCacheKey();
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return in.getCoreCacheHelper();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getCombinedCoreAndDeletesKey() {
|
||||
return in.getCombinedCoreAndDeletesKey();
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return in.getReaderCacheHelper();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -586,6 +586,16 @@ public abstract class BaseStoredFieldsFormatTestCase extends BaseIndexFileFormat
|
|||
super.document(maxDoc() - 1 - docID, visitor);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private static class DummyFilterDirectoryReader extends FilterDirectoryReader {
|
||||
|
@ -604,6 +614,11 @@ public abstract class BaseStoredFieldsFormatTestCase extends BaseIndexFileFormat
|
|||
return new DummyFilterDirectoryReader(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public void testMergeFilterReader() throws IOException {
|
||||
|
|
|
@ -175,4 +175,14 @@ public final class FieldFilterLeafReader extends FilterLeafReader {
|
|||
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -46,4 +46,9 @@ public class MismatchedDirectoryReader extends FilterDirectoryReader {
|
|||
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
|
||||
return new AssertingDirectoryReader(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return in.getReaderCacheHelper();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -45,6 +45,16 @@ public class MismatchedLeafReader extends FilterLeafReader {
|
|||
in.document(docID, new MismatchedVisitor(visitor));
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return in.getCoreCacheHelper();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return in.getReaderCacheHelper();
|
||||
}
|
||||
|
||||
static FieldInfos shuffleInfos(FieldInfos infos, Random random) {
|
||||
// first, shuffle the order
|
||||
List<FieldInfo> shuffled = new ArrayList<>();
|
||||
|
|
|
@ -156,7 +156,18 @@ public class MockRandomMergePolicy extends MergePolicy {
|
|||
if (LuceneTestCase.VERBOSE) {
|
||||
System.out.println("NOTE: MockRandomMergePolicy now swaps in a SlowCodecReaderWrapper for merging reader=" + reader);
|
||||
}
|
||||
return SlowCodecReaderWrapper.wrap(new FilterLeafReader(new MergeReaderWrapper(reader)) {});
|
||||
return SlowCodecReaderWrapper.wrap(new FilterLeafReader(new MergeReaderWrapper(reader)) {
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return in.getCoreCacheHelper();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return in.getReaderCacheHelper();
|
||||
}
|
||||
});
|
||||
} else if (thingToDo == 1) {
|
||||
// renumber fields
|
||||
// NOTE: currently this only "blocks" bulk merges just by
|
||||
|
|
|
@ -132,27 +132,6 @@ public class QueryUtils {
|
|||
}
|
||||
}
|
||||
|
||||
/** This is a MultiReader that can be used for randomly wrapping other readers
|
||||
* without creating FieldCache insanity.
|
||||
* The trick is to use an opaque/fake cache key. */
|
||||
public static class FCInvisibleMultiReader extends MultiReader {
|
||||
private final Object cacheKey = new Object();
|
||||
|
||||
public FCInvisibleMultiReader(IndexReader... readers) throws IOException {
|
||||
super(readers);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getCoreCacheKey() {
|
||||
return cacheKey;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getCombinedCoreAndDeletesKey() {
|
||||
return cacheKey;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Given an IndexSearcher, returns a new IndexSearcher whose IndexReader
|
||||
* is a MultiReader containing the Reader of the original IndexSearcher,
|
||||
|
@ -172,17 +151,17 @@ public class QueryUtils {
|
|||
IndexReader[] readers = new IndexReader[] {
|
||||
edge < 0 ? r : new MultiReader(),
|
||||
new MultiReader(),
|
||||
new FCInvisibleMultiReader(edge < 0 ? emptyReader(4) : new MultiReader(),
|
||||
new MultiReader(edge < 0 ? emptyReader(4) : new MultiReader(),
|
||||
new MultiReader(),
|
||||
0 == edge ? r : new MultiReader()),
|
||||
0 < edge ? new MultiReader() : emptyReader(7),
|
||||
new MultiReader(),
|
||||
new FCInvisibleMultiReader(0 < edge ? new MultiReader() : emptyReader(5),
|
||||
new MultiReader(0 < edge ? new MultiReader() : emptyReader(5),
|
||||
new MultiReader(),
|
||||
0 < edge ? r : new MultiReader())
|
||||
};
|
||||
|
||||
IndexSearcher out = LuceneTestCase.newSearcher(new FCInvisibleMultiReader(readers));
|
||||
IndexSearcher out = LuceneTestCase.newSearcher(new MultiReader(readers));
|
||||
out.setSimilarity(s.getSimilarity(true));
|
||||
return out;
|
||||
}
|
||||
|
@ -190,12 +169,6 @@ public class QueryUtils {
|
|||
private static IndexReader emptyReader(final int maxDoc) {
|
||||
return new LeafReader() {
|
||||
|
||||
@Override
|
||||
public void addCoreClosedListener(CoreClosedListener listener) {}
|
||||
|
||||
@Override
|
||||
public void removeCoreClosedListener(CoreClosedListener listener) {}
|
||||
|
||||
@Override
|
||||
public Fields fields() throws IOException {
|
||||
return new Fields() {
|
||||
|
@ -290,6 +263,16 @@ public class QueryUtils {
|
|||
public Sort getIndexSort() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
|
|
@ -74,7 +74,6 @@ import org.apache.lucene.document.FieldType;
|
|||
import org.apache.lucene.document.StringField;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.index.*;
|
||||
import org.apache.lucene.index.IndexReader.ReaderClosedListener;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.TermsEnum.SeekStatus;
|
||||
import org.apache.lucene.mockfile.FilterPath;
|
||||
|
@ -86,7 +85,6 @@ import org.apache.lucene.search.LRUQueryCache;
|
|||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.QueryCache;
|
||||
import org.apache.lucene.search.QueryCachingPolicy;
|
||||
import org.apache.lucene.search.QueryUtils.FCInvisibleMultiReader;
|
||||
import org.apache.lucene.store.BaseDirectoryWrapper;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.FSDirectory;
|
||||
|
@ -1664,7 +1662,7 @@ public abstract class LuceneTestCase extends Assert {
|
|||
Random random = random();
|
||||
|
||||
for (int i = 0, c = random.nextInt(6)+1; i < c; i++) {
|
||||
switch(random.nextInt(5)) {
|
||||
switch(random.nextInt(4)) {
|
||||
case 0:
|
||||
// will create no FC insanity in atomic case, as ParallelLeafReader has own cache key:
|
||||
if (VERBOSE) {
|
||||
|
@ -1675,15 +1673,6 @@ public abstract class LuceneTestCase extends Assert {
|
|||
new ParallelCompositeReader((CompositeReader) r);
|
||||
break;
|
||||
case 1:
|
||||
// Häckidy-Hick-Hack: a standard MultiReader will cause FC insanity, so we use
|
||||
// QueryUtils' reader with a fake cache key, so insanity checker cannot walk
|
||||
// along our reader:
|
||||
if (VERBOSE) {
|
||||
System.out.println("NOTE: LuceneTestCase.wrapReader: wrapping previous reader=" + r + " with FCInvisibleMultiReader");
|
||||
}
|
||||
r = new FCInvisibleMultiReader(r);
|
||||
break;
|
||||
case 2:
|
||||
if (r instanceof LeafReader) {
|
||||
final LeafReader ar = (LeafReader) r;
|
||||
final List<String> allFields = new ArrayList<>();
|
||||
|
@ -1703,7 +1692,7 @@ public abstract class LuceneTestCase extends Assert {
|
|||
);
|
||||
}
|
||||
break;
|
||||
case 3:
|
||||
case 2:
|
||||
// Häckidy-Hick-Hack: a standard Reader will cause FC insanity, so we use
|
||||
// QueryUtils' reader with a fake cache key, so insanity checker cannot walk
|
||||
// along our reader:
|
||||
|
@ -1716,7 +1705,7 @@ public abstract class LuceneTestCase extends Assert {
|
|||
r = new AssertingDirectoryReader((DirectoryReader)r);
|
||||
}
|
||||
break;
|
||||
case 4:
|
||||
case 3:
|
||||
if (VERBOSE) {
|
||||
System.out.println("NOTE: LuceneTestCase.wrapReader: wrapping previous reader=" + r + " with MismatchedLeaf/DirectoryReader");
|
||||
}
|
||||
|
@ -1731,10 +1720,6 @@ public abstract class LuceneTestCase extends Assert {
|
|||
}
|
||||
}
|
||||
|
||||
if ((r instanceof CompositeReader) && !(r instanceof FCInvisibleMultiReader)) {
|
||||
// prevent cache insanity caused by e.g. ParallelCompositeReader, to fix we wrap one more time:
|
||||
r = new FCInvisibleMultiReader(r);
|
||||
}
|
||||
if (VERBOSE) {
|
||||
System.out.println("wrapReader wrapped: " +r);
|
||||
}
|
||||
|
@ -1900,7 +1885,7 @@ public abstract class LuceneTestCase extends Assert {
|
|||
} else {
|
||||
int threads = 0;
|
||||
final ThreadPoolExecutor ex;
|
||||
if (random.nextBoolean()) {
|
||||
if (r.getReaderCacheHelper() == null || random.nextBoolean()) {
|
||||
ex = null;
|
||||
} else {
|
||||
threads = TestUtil.nextInt(random, 1, 8);
|
||||
|
@ -1914,12 +1899,7 @@ public abstract class LuceneTestCase extends Assert {
|
|||
if (VERBOSE) {
|
||||
System.out.println("NOTE: newSearcher using ExecutorService with " + threads + " threads");
|
||||
}
|
||||
r.addReaderClosedListener(new ReaderClosedListener() {
|
||||
@Override
|
||||
public void onClose(IndexReader reader) {
|
||||
TestUtil.shutdownExecutorService(ex);
|
||||
}
|
||||
});
|
||||
r.getReaderCacheHelper().addClosedListener(cacheKey -> TestUtil.shutdownExecutorService(ex));
|
||||
}
|
||||
IndexSearcher ret;
|
||||
if (wrapWithAssertions) {
|
||||
|
|
|
@ -63,6 +63,7 @@ import org.apache.lucene.analysis.util.ResourceLoader;
|
|||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.IndexDeletionPolicy;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
|
@ -219,7 +220,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
|
|||
|
||||
public Date getStartTimeStamp() { return startTime; }
|
||||
|
||||
private final Map<Object, IndexFingerprint> perSegmentFingerprintCache = new MapMaker().weakKeys().makeMap();
|
||||
private final Map<IndexReader.CacheKey, IndexFingerprint> perSegmentFingerprintCache = new MapMaker().weakKeys().makeMap();
|
||||
|
||||
public long getStartNanoTime() {
|
||||
return startNanoTime;
|
||||
|
@ -1775,8 +1776,14 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
|
|||
*/
|
||||
public IndexFingerprint getIndexFingerprint(SolrIndexSearcher searcher, LeafReaderContext ctx, long maxVersion)
|
||||
throws IOException {
|
||||
IndexReader.CacheHelper cacheHelper = ctx.reader().getReaderCacheHelper();
|
||||
if (cacheHelper == null) {
|
||||
log.debug("Cannot cache IndexFingerprint as reader does not support caching. searcher:{} reader:{} readerHash:{} maxVersion:{}", searcher, ctx.reader(), ctx.reader().hashCode(), maxVersion);
|
||||
return IndexFingerprint.getFingerprint(searcher, ctx, maxVersion);
|
||||
}
|
||||
|
||||
IndexFingerprint f = null;
|
||||
f = perSegmentFingerprintCache.get(ctx.reader().getCombinedCoreAndDeletesKey());
|
||||
f = perSegmentFingerprintCache.get(cacheHelper.getKey());
|
||||
// fingerprint is either not cached or
|
||||
// if we want fingerprint only up to a version less than maxVersionEncountered in the segment, or
|
||||
// documents were deleted from segment for which fingerprint was cached
|
||||
|
@ -1787,7 +1794,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
|
|||
// cache fingerprint for the segment only if all the versions in the segment are included in the fingerprint
|
||||
if (f.getMaxVersionEncountered() == f.getMaxInHash()) {
|
||||
log.info("Caching fingerprint for searcher:{} leafReaderContext:{} mavVersion:{}", searcher, ctx, maxVersion);
|
||||
perSegmentFingerprintCache.put(ctx.reader().getCombinedCoreAndDeletesKey(), f);
|
||||
perSegmentFingerprintCache.put(cacheHelper.getKey(), f);
|
||||
}
|
||||
|
||||
} else {
|
||||
|
|
|
@ -761,6 +761,8 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
|
|||
}
|
||||
}
|
||||
|
||||
// this reader alters the content of the given reader so it should not
|
||||
// delegate the caching stuff
|
||||
private class ReaderWrapper extends FilterLeafReader {
|
||||
|
||||
private String field;
|
||||
|
@ -774,10 +776,6 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
|
|||
return null;
|
||||
}
|
||||
|
||||
public Object getCoreCacheKey() {
|
||||
return in.getCoreCacheKey();
|
||||
}
|
||||
|
||||
public FieldInfos getFieldInfos() {
|
||||
Iterator<FieldInfo> it = in.getFieldInfos().iterator();
|
||||
List<FieldInfo> newInfos = new ArrayList<>();
|
||||
|
@ -805,6 +803,21 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
|
|||
FieldInfos infos = new FieldInfos(newInfos.toArray(new FieldInfo[newInfos.size()]));
|
||||
return infos;
|
||||
}
|
||||
|
||||
// NOTE: delegating the caches is wrong here as we are altering the content
|
||||
// of the reader, this should ONLY be used under an uninvertingreader which
|
||||
// will restore doc values back using uninversion, otherwise all sorts of
|
||||
// crazy things could happen.
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return in.getCoreCacheHelper();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return in.getReaderCacheHelper();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -914,4 +914,14 @@ class TermVectorReusingLeafReader extends FilterLeafReader {
|
|||
return tvFields;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -47,7 +47,6 @@ public final class SlowCompositeReaderWrapper extends LeafReader {
|
|||
|
||||
private final CompositeReader in;
|
||||
private final Fields fields;
|
||||
private final boolean merging;
|
||||
|
||||
/** This method is sugar for getting an {@link LeafReader} from
|
||||
* an {@link IndexReader} of any kind. If the reader is already atomic,
|
||||
|
@ -55,19 +54,18 @@ public final class SlowCompositeReaderWrapper extends LeafReader {
|
|||
*/
|
||||
public static LeafReader wrap(IndexReader reader) throws IOException {
|
||||
if (reader instanceof CompositeReader) {
|
||||
return new SlowCompositeReaderWrapper((CompositeReader) reader, false);
|
||||
return new SlowCompositeReaderWrapper((CompositeReader) reader);
|
||||
} else {
|
||||
assert reader instanceof LeafReader;
|
||||
return (LeafReader) reader;
|
||||
}
|
||||
}
|
||||
|
||||
SlowCompositeReaderWrapper(CompositeReader reader, boolean merging) throws IOException {
|
||||
SlowCompositeReaderWrapper(CompositeReader reader) throws IOException {
|
||||
super();
|
||||
in = reader;
|
||||
fields = MultiFields.getFields(in);
|
||||
in.registerParentReader(this);
|
||||
this.merging = merging;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -76,13 +74,16 @@ public final class SlowCompositeReaderWrapper extends LeafReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void addCoreClosedListener(CoreClosedListener listener) {
|
||||
addCoreClosedListenerAsReaderClosedListener(in, listener);
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return in.getReaderCacheHelper();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeCoreClosedListener(CoreClosedListener listener) {
|
||||
removeCoreClosedListenerAsReaderClosedListener(in, listener);
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
// TODO: this is trappy as the expectation is that core keys live for a long
|
||||
// time, but here we need to bound it to the lifetime of the wrapped
|
||||
// composite reader? Unfortunately some features seem to rely on this...
|
||||
return in.getReaderCacheHelper();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -120,7 +121,8 @@ public final class SlowCompositeReaderWrapper extends LeafReader {
|
|||
SortedDocValues dv = MultiDocValues.getSortedValues(in, field);
|
||||
if (dv instanceof MultiSortedDocValues) {
|
||||
map = ((MultiSortedDocValues)dv).mapping;
|
||||
if (map.owner == getCoreCacheKey() && merging == false) {
|
||||
IndexReader.CacheHelper cacheHelper = getReaderCacheHelper();
|
||||
if (cacheHelper != null && map.owner == cacheHelper.getKey()) {
|
||||
cachedOrdMaps.put(field, map);
|
||||
}
|
||||
}
|
||||
|
@ -161,7 +163,8 @@ public final class SlowCompositeReaderWrapper extends LeafReader {
|
|||
SortedSetDocValues dv = MultiDocValues.getSortedSetValues(in, field);
|
||||
if (dv instanceof MultiDocValues.MultiSortedSetDocValues) {
|
||||
map = ((MultiDocValues.MultiSortedSetDocValues)dv).mapping;
|
||||
if (map.owner == getCoreCacheKey() && merging == false) {
|
||||
IndexReader.CacheHelper cacheHelper = getReaderCacheHelper();
|
||||
if (cacheHelper != null && map.owner == cacheHelper.getKey()) {
|
||||
cachedOrdMaps.put(field, map);
|
||||
}
|
||||
}
|
||||
|
@ -195,7 +198,7 @@ public final class SlowCompositeReaderWrapper extends LeafReader {
|
|||
|
||||
// TODO: this could really be a weak map somewhere else on the coreCacheKey,
|
||||
// but do we really need to optimize slow-wrapper any more?
|
||||
private final Map<String,OrdinalMap> cachedOrdMaps = new HashMap<>();
|
||||
final Map<String,OrdinalMap> cachedOrdMaps = new HashMap<>();
|
||||
|
||||
@Override
|
||||
public NumericDocValues getNormValues(String field) throws IOException {
|
||||
|
@ -245,16 +248,6 @@ public final class SlowCompositeReaderWrapper extends LeafReader {
|
|||
return MultiFields.getMergedFieldInfos(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getCoreCacheKey() {
|
||||
return in.getCoreCacheKey();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getCombinedCoreAndDeletesKey() {
|
||||
return in.getCombinedCoreAndDeletesKey();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doClose() throws IOException {
|
||||
// TODO: as this is a wrapper, should we really close the delegate?
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.util.HashMap;
|
|||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.queries.function.FunctionValues;
|
||||
import org.apache.lucene.queries.function.ValueSource;
|
||||
|
@ -162,7 +163,11 @@ public class RptWithGeometrySpatialField extends AbstractSpatialFieldType<Compos
|
|||
}
|
||||
docId = doc;
|
||||
//lookup in cache
|
||||
PerSegCacheKey key = new PerSegCacheKey(readerContext.reader().getCoreCacheKey(), doc);
|
||||
IndexReader.CacheHelper cacheHelper = readerContext.reader().getCoreCacheHelper();
|
||||
if (cacheHelper == null) {
|
||||
throw new IllegalStateException("Leaf " + readerContext.reader() + " is not suited for caching");
|
||||
}
|
||||
PerSegCacheKey key = new PerSegCacheKey(cacheHelper.getKey(), doc);
|
||||
shape = cache.get(key);
|
||||
if (shape == null) {
|
||||
shape = (Shape) targetFuncValues.objectVal(doc);
|
||||
|
|
|
@ -388,12 +388,23 @@ public class CollapsingQParserPlugin extends QParserPlugin {
|
|||
this.field = field;
|
||||
}
|
||||
|
||||
public SortedDocValues getSortedDocValues(String field) {
|
||||
return null;
|
||||
// NOTE: delegating the caches is wrong here as we are altering the content
|
||||
// of the reader, this should ONLY be used under an uninvertingreader which
|
||||
// will restore doc values back using uninversion, otherwise all sorts of
|
||||
// crazy things could happen.
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return in.getCoreCacheHelper();
|
||||
}
|
||||
|
||||
public Object getCoreCacheKey() {
|
||||
return in.getCoreCacheKey();
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return in.getReaderCacheHelper();
|
||||
}
|
||||
|
||||
public SortedDocValues getSortedDocValues(String field) {
|
||||
return null;
|
||||
}
|
||||
|
||||
public FieldInfos getFieldInfos() {
|
||||
|
|
|
@ -118,13 +118,14 @@ public class Insanity {
|
|||
// important to override these, so fieldcaches are shared on what we wrap
|
||||
|
||||
@Override
|
||||
public Object getCoreCacheKey() {
|
||||
return in.getCoreCacheKey();
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return in.getCoreCacheHelper();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getCombinedCoreAndDeletesKey() {
|
||||
return in.getCombinedCoreAndDeletesKey();
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return in.getReaderCacheHelper();
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,7 +17,6 @@
|
|||
package org.apache.solr.uninverting;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
|
||||
import org.apache.lucene.document.NumericDocValuesField;
|
||||
import org.apache.lucene.index.BinaryDocValues;
|
||||
|
@ -41,7 +40,6 @@ import org.apache.lucene.util.RamUsageEstimator;
|
|||
* <p>Created: May 19, 2004 11:13:14 AM
|
||||
*
|
||||
* @since lucene 1.4
|
||||
* @see FieldCacheSanityChecker
|
||||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
|
@ -357,7 +355,7 @@ public interface FieldCache {
|
|||
private final Object custom;
|
||||
private final Accountable value;
|
||||
|
||||
public CacheEntry(Object readerKey, String fieldName,
|
||||
public CacheEntry(IndexReader.CacheKey readerKey, String fieldName,
|
||||
Class<?> cacheType,
|
||||
Object custom,
|
||||
Accountable value) {
|
||||
|
@ -437,21 +435,13 @@ public interface FieldCache {
|
|||
|
||||
/**
|
||||
* Expert: drops all cache entries associated with this
|
||||
* reader {@link IndexReader#getCoreCacheKey}. NOTE: this cache key must
|
||||
* reader {@link org.apache.lucene.index.IndexReader.CacheHelper#getKey()}.
|
||||
* NOTE: this cache key must
|
||||
* precisely match the reader that the cache entry is
|
||||
* keyed on. If you pass a top-level reader, it usually
|
||||
* will have no effect as Lucene now caches at the segment
|
||||
* reader level.
|
||||
*/
|
||||
public void purgeByCacheKey(Object coreCacheKey);
|
||||
public void purgeByCacheKey(IndexReader.CacheKey coreCacheKey);
|
||||
|
||||
/**
|
||||
* If non-null, FieldCacheImpl will warn whenever
|
||||
* entries are created that are not sane according to
|
||||
* {@link FieldCacheSanityChecker}.
|
||||
*/
|
||||
public void setInfoStream(PrintStream stream);
|
||||
|
||||
/** counterpart of {@link #setInfoStream(PrintStream)} */
|
||||
public PrintStream getInfoStream();
|
||||
}
|
||||
|
|
|
@ -17,7 +17,6 @@
|
|||
package org.apache.solr.uninverting;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
|
@ -31,13 +30,13 @@ import org.apache.lucene.index.DocValues;
|
|||
import org.apache.lucene.index.DocValuesType;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexOptions;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.NumericDocValues;
|
||||
import org.apache.lucene.index.PointValues;
|
||||
import org.apache.lucene.index.PointValues.IntersectVisitor;
|
||||
import org.apache.lucene.index.PointValues.Relation;
|
||||
import org.apache.lucene.index.PostingsEnum;
|
||||
import org.apache.lucene.index.SegmentReader;
|
||||
import org.apache.lucene.index.SortedDocValues;
|
||||
import org.apache.lucene.index.SortedSetDocValues;
|
||||
import org.apache.lucene.index.Terms;
|
||||
|
@ -82,7 +81,7 @@ public class FieldCacheImpl implements FieldCache {
|
|||
}
|
||||
|
||||
@Override
|
||||
public synchronized void purgeByCacheKey(Object coreCacheKey) {
|
||||
public synchronized void purgeByCacheKey(IndexReader.CacheKey coreCacheKey) {
|
||||
for(Cache c : caches.values()) {
|
||||
c.purgeByCacheKey(coreCacheKey);
|
||||
}
|
||||
|
@ -95,8 +94,8 @@ public class FieldCacheImpl implements FieldCache {
|
|||
final Cache cache = cacheEntry.getValue();
|
||||
final Class<?> cacheType = cacheEntry.getKey();
|
||||
synchronized(cache.readerCache) {
|
||||
for (final Map.Entry<Object,Map<CacheKey, Accountable>> readerCacheEntry : cache.readerCache.entrySet()) {
|
||||
final Object readerKey = readerCacheEntry.getKey();
|
||||
for (final Map.Entry<IndexReader.CacheKey,Map<CacheKey, Accountable>> readerCacheEntry : cache.readerCache.entrySet()) {
|
||||
final IndexReader.CacheKey readerKey = readerCacheEntry.getKey();
|
||||
if (readerKey == null) continue;
|
||||
final Map<CacheKey, Accountable> innerCache = readerCacheEntry.getValue();
|
||||
for (final Map.Entry<CacheKey, Accountable> mapEntry : innerCache.entrySet()) {
|
||||
|
@ -112,10 +111,14 @@ public class FieldCacheImpl implements FieldCache {
|
|||
}
|
||||
|
||||
// per-segment fieldcaches don't purge until the shared core closes.
|
||||
final SegmentReader.CoreClosedListener purgeCore = FieldCacheImpl.this::purgeByCacheKey;
|
||||
final IndexReader.ClosedListener purgeCore = FieldCacheImpl.this::purgeByCacheKey;
|
||||
|
||||
private void initReader(LeafReader reader) {
|
||||
reader.addCoreClosedListener(purgeCore);
|
||||
IndexReader.CacheHelper cacheHelper = reader.getCoreCacheHelper();
|
||||
if (cacheHelper == null) {
|
||||
throw new IllegalStateException("Cannot cache on " + reader);
|
||||
}
|
||||
cacheHelper.addClosedListener(purgeCore);
|
||||
}
|
||||
|
||||
/** Expert: Internal cache. */
|
||||
|
@ -127,13 +130,13 @@ public class FieldCacheImpl implements FieldCache {
|
|||
|
||||
final FieldCacheImpl wrapper;
|
||||
|
||||
final Map<Object,Map<CacheKey,Accountable>> readerCache = new WeakHashMap<>();
|
||||
final Map<IndexReader.CacheKey,Map<CacheKey,Accountable>> readerCache = new WeakHashMap<>();
|
||||
|
||||
protected abstract Accountable createValue(LeafReader reader, CacheKey key)
|
||||
throws IOException;
|
||||
|
||||
/** Remove this reader from the cache, if present. */
|
||||
public void purgeByCacheKey(Object coreCacheKey) {
|
||||
public void purgeByCacheKey(IndexReader.CacheKey coreCacheKey) {
|
||||
synchronized(readerCache) {
|
||||
readerCache.remove(coreCacheKey);
|
||||
}
|
||||
|
@ -142,7 +145,11 @@ public class FieldCacheImpl implements FieldCache {
|
|||
/** Sets the key to the value for the provided reader;
|
||||
* if the key is already set then this doesn't change it. */
|
||||
public void put(LeafReader reader, CacheKey key, Accountable value) {
|
||||
final Object readerKey = reader.getCoreCacheKey();
|
||||
IndexReader.CacheHelper cacheHelper = reader.getCoreCacheHelper();
|
||||
if (cacheHelper == null) {
|
||||
throw new IllegalStateException("Cannot cache on " + reader);
|
||||
}
|
||||
final IndexReader.CacheKey readerKey = cacheHelper.getKey();
|
||||
synchronized (readerCache) {
|
||||
Map<CacheKey,Accountable> innerCache = readerCache.get(readerKey);
|
||||
if (innerCache == null) {
|
||||
|
@ -163,7 +170,12 @@ public class FieldCacheImpl implements FieldCache {
|
|||
public Object get(LeafReader reader, CacheKey key) throws IOException {
|
||||
Map<CacheKey,Accountable> innerCache;
|
||||
Accountable value;
|
||||
final Object readerKey = reader.getCoreCacheKey();
|
||||
IndexReader.CacheHelper cacheHelper = reader.getCoreCacheHelper();
|
||||
if (cacheHelper == null) {
|
||||
reader.getCoreCacheHelper();
|
||||
throw new IllegalStateException("Cannot cache on " + reader);
|
||||
}
|
||||
final IndexReader.CacheKey readerKey = cacheHelper.getKey();
|
||||
synchronized (readerCache) {
|
||||
innerCache = readerCache.get(readerKey);
|
||||
if (innerCache == null) {
|
||||
|
@ -188,39 +200,12 @@ public class FieldCacheImpl implements FieldCache {
|
|||
synchronized (readerCache) {
|
||||
innerCache.put(key, progress.value);
|
||||
}
|
||||
|
||||
// Only check if key.custom (the parser) is
|
||||
// non-null; else, we check twice for a single
|
||||
// call to FieldCache.getXXX
|
||||
if (key.custom != null && wrapper != null) {
|
||||
final PrintStream infoStream = wrapper.getInfoStream();
|
||||
if (infoStream != null) {
|
||||
printNewInsanity(infoStream, progress.value);
|
||||
}
|
||||
}
|
||||
}
|
||||
return progress.value;
|
||||
}
|
||||
}
|
||||
return value;
|
||||
}
|
||||
|
||||
private void printNewInsanity(PrintStream infoStream, Object value) {
|
||||
final FieldCacheSanityChecker.Insanity[] insanities = FieldCacheSanityChecker.checkSanity(wrapper);
|
||||
for(int i=0;i<insanities.length;i++) {
|
||||
final FieldCacheSanityChecker.Insanity insanity = insanities[i];
|
||||
final CacheEntry[] entries = insanity.getCacheEntries();
|
||||
for(int j=0;j<entries.length;j++) {
|
||||
if (entries[j].getValue() == value) {
|
||||
// OK this insanity involves our entry
|
||||
infoStream.println("WARNING: new FieldCache insanity created\nDetails: " + insanity.toString());
|
||||
infoStream.println("\nStack:\n");
|
||||
new Throwable().printStackTrace(infoStream);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Expert: Every composite-key in the internal cache is of this type. */
|
||||
|
@ -1265,14 +1250,5 @@ public class FieldCacheImpl implements FieldCache {
|
|||
}
|
||||
}
|
||||
|
||||
private volatile PrintStream infoStream;
|
||||
|
||||
public void setInfoStream(PrintStream stream) {
|
||||
infoStream = stream;
|
||||
}
|
||||
|
||||
public PrintStream getInfoStream() {
|
||||
return infoStream;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -1,426 +0,0 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.solr.uninverting;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexReaderContext;
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.MapOfSets;
|
||||
import org.apache.solr.uninverting.FieldCache.CacheEntry;
|
||||
|
||||
/**
|
||||
* Provides methods for sanity checking that entries in the FieldCache
|
||||
* are not wasteful or inconsistent.
|
||||
* </p>
|
||||
* <p>
|
||||
* Lucene 2.9 Introduced numerous enhancements into how the FieldCache
|
||||
* is used by the low levels of Lucene searching (for Sorting and
|
||||
* ValueSourceQueries) to improve both the speed for Sorting, as well
|
||||
* as reopening of IndexReaders. But these changes have shifted the
|
||||
* usage of FieldCache from "top level" IndexReaders (frequently a
|
||||
* MultiReader or DirectoryReader) down to the leaf level SegmentReaders.
|
||||
* As a result, existing applications that directly access the FieldCache
|
||||
* may find RAM usage increase significantly when upgrading to 2.9 or
|
||||
* Later. This class provides an API for these applications (or their
|
||||
* Unit tests) to check at run time if the FieldCache contains "insane"
|
||||
* usages of the FieldCache.
|
||||
* </p>
|
||||
* @lucene.experimental
|
||||
* @see FieldCache
|
||||
* @see FieldCacheSanityChecker.Insanity
|
||||
* @see FieldCacheSanityChecker.InsanityType
|
||||
*/
|
||||
final class FieldCacheSanityChecker {
|
||||
|
||||
public FieldCacheSanityChecker() {
|
||||
/* NOOP */
|
||||
}
|
||||
|
||||
/**
|
||||
* Quick and dirty convenience method
|
||||
* @see #check
|
||||
*/
|
||||
public static Insanity[] checkSanity(FieldCache cache) {
|
||||
return checkSanity(cache.getCacheEntries());
|
||||
}
|
||||
|
||||
/**
|
||||
* Quick and dirty convenience method that instantiates an instance with
|
||||
* "good defaults" and uses it to test the CacheEntrys
|
||||
* @see #check
|
||||
*/
|
||||
public static Insanity[] checkSanity(CacheEntry... cacheEntries) {
|
||||
FieldCacheSanityChecker sanityChecker = new FieldCacheSanityChecker();
|
||||
return sanityChecker.check(cacheEntries);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Tests a CacheEntry[] for indication of "insane" cache usage.
|
||||
* <p>
|
||||
* <B>NOTE:</b>FieldCache CreationPlaceholder objects are ignored.
|
||||
* (:TODO: is this a bad idea? are we masking a real problem?)
|
||||
* </p>
|
||||
*/
|
||||
public Insanity[] check(CacheEntry... cacheEntries) {
|
||||
if (null == cacheEntries || 0 == cacheEntries.length)
|
||||
return new Insanity[0];
|
||||
|
||||
// the indirect mapping lets MapOfSet dedup identical valIds for us
|
||||
//
|
||||
// maps the (valId) identityhashCode of cache values to
|
||||
// sets of CacheEntry instances
|
||||
final MapOfSets<Integer, CacheEntry> valIdToItems = new MapOfSets<>(new HashMap<Integer, Set<CacheEntry>>(17));
|
||||
// maps ReaderField keys to Sets of ValueIds
|
||||
final MapOfSets<ReaderField, Integer> readerFieldToValIds = new MapOfSets<>(new HashMap<ReaderField, Set<Integer>>(17));
|
||||
//
|
||||
|
||||
// any keys that we know result in more then one valId
|
||||
final Set<ReaderField> valMismatchKeys = new HashSet<>();
|
||||
|
||||
// iterate over all the cacheEntries to get the mappings we'll need
|
||||
for (int i = 0; i < cacheEntries.length; i++) {
|
||||
final CacheEntry item = cacheEntries[i];
|
||||
final Accountable val = item.getValue();
|
||||
|
||||
// It's OK to have dup entries, where one is eg
|
||||
// float[] and the other is the Bits (from
|
||||
// getDocWithField())
|
||||
if (val instanceof FieldCacheImpl.BitsEntry) {
|
||||
continue;
|
||||
}
|
||||
|
||||
if (val instanceof FieldCache.CreationPlaceholder)
|
||||
continue;
|
||||
|
||||
final ReaderField rf = new ReaderField(item.getReaderKey(),
|
||||
item.getFieldName());
|
||||
|
||||
final Integer valId = Integer.valueOf(System.identityHashCode(val));
|
||||
|
||||
// indirect mapping, so the MapOfSet will dedup identical valIds for us
|
||||
valIdToItems.put(valId, item);
|
||||
if (1 < readerFieldToValIds.put(rf, valId)) {
|
||||
valMismatchKeys.add(rf);
|
||||
}
|
||||
}
|
||||
|
||||
final List<Insanity> insanity = new ArrayList<>(valMismatchKeys.size() * 3);
|
||||
|
||||
insanity.addAll(checkValueMismatch(valIdToItems,
|
||||
readerFieldToValIds,
|
||||
valMismatchKeys));
|
||||
insanity.addAll(checkSubreaders(valIdToItems,
|
||||
readerFieldToValIds));
|
||||
|
||||
return insanity.toArray(new Insanity[insanity.size()]);
|
||||
}
|
||||
|
||||
/**
|
||||
* Internal helper method used by check that iterates over
|
||||
* valMismatchKeys and generates a Collection of Insanity
|
||||
* instances accordingly. The MapOfSets are used to populate
|
||||
* the Insanity objects.
|
||||
* @see InsanityType#VALUEMISMATCH
|
||||
*/
|
||||
private Collection<Insanity> checkValueMismatch(MapOfSets<Integer, CacheEntry> valIdToItems,
|
||||
MapOfSets<ReaderField, Integer> readerFieldToValIds,
|
||||
Set<ReaderField> valMismatchKeys) {
|
||||
|
||||
final List<Insanity> insanity = new ArrayList<>(valMismatchKeys.size() * 3);
|
||||
|
||||
if (! valMismatchKeys.isEmpty() ) {
|
||||
// we have multiple values for some ReaderFields
|
||||
|
||||
final Map<ReaderField, Set<Integer>> rfMap = readerFieldToValIds.getMap();
|
||||
final Map<Integer, Set<CacheEntry>> valMap = valIdToItems.getMap();
|
||||
for (final ReaderField rf : valMismatchKeys) {
|
||||
final List<CacheEntry> badEntries = new ArrayList<>(valMismatchKeys.size() * 2);
|
||||
for(final Integer value: rfMap.get(rf)) {
|
||||
for (final CacheEntry cacheEntry : valMap.get(value)) {
|
||||
badEntries.add(cacheEntry);
|
||||
}
|
||||
}
|
||||
|
||||
CacheEntry[] badness = new CacheEntry[badEntries.size()];
|
||||
badness = badEntries.toArray(badness);
|
||||
|
||||
insanity.add(new Insanity(InsanityType.VALUEMISMATCH,
|
||||
"Multiple distinct value objects for " +
|
||||
rf.toString(), badness));
|
||||
}
|
||||
}
|
||||
return insanity;
|
||||
}
|
||||
|
||||
/**
|
||||
* Internal helper method used by check that iterates over
|
||||
* the keys of readerFieldToValIds and generates a Collection
|
||||
* of Insanity instances whenever two (or more) ReaderField instances are
|
||||
* found that have an ancestry relationships.
|
||||
*
|
||||
* @see InsanityType#SUBREADER
|
||||
*/
|
||||
private Collection<Insanity> checkSubreaders( MapOfSets<Integer, CacheEntry> valIdToItems,
|
||||
MapOfSets<ReaderField, Integer> readerFieldToValIds) {
|
||||
|
||||
final List<Insanity> insanity = new ArrayList<>(23);
|
||||
|
||||
Map<ReaderField, Set<ReaderField>> badChildren = new HashMap<>(17);
|
||||
MapOfSets<ReaderField, ReaderField> badKids = new MapOfSets<>(badChildren); // wrapper
|
||||
|
||||
Map<Integer, Set<CacheEntry>> viToItemSets = valIdToItems.getMap();
|
||||
Map<ReaderField, Set<Integer>> rfToValIdSets = readerFieldToValIds.getMap();
|
||||
|
||||
Set<ReaderField> seen = new HashSet<>(17);
|
||||
|
||||
Set<ReaderField> readerFields = rfToValIdSets.keySet();
|
||||
for (final ReaderField rf : readerFields) {
|
||||
|
||||
if (seen.contains(rf)) continue;
|
||||
|
||||
List<Object> kids = getAllDescendantReaderKeys(rf.readerKey);
|
||||
for (Object kidKey : kids) {
|
||||
ReaderField kid = new ReaderField(kidKey, rf.fieldName);
|
||||
|
||||
if (badChildren.containsKey(kid)) {
|
||||
// we've already process this kid as RF and found other problems
|
||||
// track those problems as our own
|
||||
badKids.put(rf, kid);
|
||||
badKids.putAll(rf, badChildren.get(kid));
|
||||
badChildren.remove(kid);
|
||||
|
||||
} else if (rfToValIdSets.containsKey(kid)) {
|
||||
// we have cache entries for the kid
|
||||
badKids.put(rf, kid);
|
||||
}
|
||||
seen.add(kid);
|
||||
}
|
||||
seen.add(rf);
|
||||
}
|
||||
|
||||
// every mapping in badKids represents an Insanity
|
||||
for (final ReaderField parent : badChildren.keySet()) {
|
||||
Set<ReaderField> kids = badChildren.get(parent);
|
||||
|
||||
List<CacheEntry> badEntries = new ArrayList<>(kids.size() * 2);
|
||||
|
||||
// put parent entr(ies) in first
|
||||
{
|
||||
for (final Integer value : rfToValIdSets.get(parent)) {
|
||||
badEntries.addAll(viToItemSets.get(value));
|
||||
}
|
||||
}
|
||||
|
||||
// now the entries for the descendants
|
||||
for (final ReaderField kid : kids) {
|
||||
for (final Integer value : rfToValIdSets.get(kid)) {
|
||||
badEntries.addAll(viToItemSets.get(value));
|
||||
}
|
||||
}
|
||||
|
||||
CacheEntry[] badness = new CacheEntry[badEntries.size()];
|
||||
badness = badEntries.toArray(badness);
|
||||
|
||||
insanity.add(new Insanity(InsanityType.SUBREADER,
|
||||
"Found caches for descendants of " +
|
||||
parent.toString(),
|
||||
badness));
|
||||
}
|
||||
|
||||
return insanity;
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if the seed is an IndexReader, and if so will walk
|
||||
* the hierarchy of subReaders building up a list of the objects
|
||||
* returned by {@code seed.getCoreCacheKey()}
|
||||
*/
|
||||
private List<Object> getAllDescendantReaderKeys(Object seed) {
|
||||
List<Object> all = new ArrayList<>(17); // will grow as we iter
|
||||
all.add(seed);
|
||||
for (int i = 0; i < all.size(); i++) {
|
||||
final Object obj = all.get(i);
|
||||
// TODO: We don't check closed readers here (as getTopReaderContext
|
||||
// throws AlreadyClosedException), what should we do? Reflection?
|
||||
if (obj instanceof IndexReader) {
|
||||
try {
|
||||
final List<IndexReaderContext> childs =
|
||||
((IndexReader) obj).getContext().children();
|
||||
if (childs != null) { // it is composite reader
|
||||
for (final IndexReaderContext ctx : childs) {
|
||||
all.add(ctx.reader().getCoreCacheKey());
|
||||
}
|
||||
}
|
||||
} catch (AlreadyClosedException ace) {
|
||||
// ignore this reader
|
||||
}
|
||||
}
|
||||
}
|
||||
// need to skip the first, because it was the seed
|
||||
return all.subList(1, all.size());
|
||||
}
|
||||
|
||||
/**
|
||||
* Simple pair object for using "readerKey + fieldName" a Map key
|
||||
*/
|
||||
private final static class ReaderField {
|
||||
public final Object readerKey;
|
||||
public final String fieldName;
|
||||
public ReaderField(Object readerKey, String fieldName) {
|
||||
this.readerKey = readerKey;
|
||||
this.fieldName = fieldName;
|
||||
}
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return System.identityHashCode(readerKey) * fieldName.hashCode();
|
||||
}
|
||||
@Override
|
||||
public boolean equals(Object that) {
|
||||
if (! (that instanceof ReaderField)) return false;
|
||||
|
||||
ReaderField other = (ReaderField) that;
|
||||
return (this.readerKey == other.readerKey &&
|
||||
this.fieldName.equals(other.fieldName));
|
||||
}
|
||||
@Override
|
||||
public String toString() {
|
||||
return readerKey.toString() + "+" + fieldName;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Simple container for a collection of related CacheEntry objects that
|
||||
* in conjunction with each other represent some "insane" usage of the
|
||||
* FieldCache.
|
||||
*/
|
||||
public final static class Insanity {
|
||||
private final InsanityType type;
|
||||
private final String msg;
|
||||
private final CacheEntry[] entries;
|
||||
public Insanity(InsanityType type, String msg, CacheEntry... entries) {
|
||||
if (null == type) {
|
||||
throw new IllegalArgumentException
|
||||
("Insanity requires non-null InsanityType");
|
||||
}
|
||||
if (null == entries || 0 == entries.length) {
|
||||
throw new IllegalArgumentException
|
||||
("Insanity requires non-null/non-empty CacheEntry[]");
|
||||
}
|
||||
this.type = type;
|
||||
this.msg = msg;
|
||||
this.entries = entries;
|
||||
|
||||
}
|
||||
/**
|
||||
* Type of insane behavior this object represents
|
||||
*/
|
||||
public InsanityType getType() { return type; }
|
||||
/**
|
||||
* Description of hte insane behavior
|
||||
*/
|
||||
public String getMsg() { return msg; }
|
||||
/**
|
||||
* CacheEntry objects which suggest a problem
|
||||
*/
|
||||
public CacheEntry[] getCacheEntries() { return entries; }
|
||||
/**
|
||||
* Multi-Line representation of this Insanity object, starting with
|
||||
* the Type and Msg, followed by each CacheEntry.toString() on its
|
||||
* own line prefaced by a tab character
|
||||
*/
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder buf = new StringBuilder();
|
||||
buf.append(getType()).append(": ");
|
||||
|
||||
String m = getMsg();
|
||||
if (null != m) buf.append(m);
|
||||
|
||||
buf.append('\n');
|
||||
|
||||
CacheEntry[] ce = getCacheEntries();
|
||||
for (int i = 0; i < ce.length; i++) {
|
||||
buf.append('\t').append(ce[i].toString()).append('\n');
|
||||
}
|
||||
|
||||
return buf.toString();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* An Enumeration of the different types of "insane" behavior that
|
||||
* may be detected in a FieldCache.
|
||||
*
|
||||
* @see InsanityType#SUBREADER
|
||||
* @see InsanityType#VALUEMISMATCH
|
||||
* @see InsanityType#EXPECTED
|
||||
*/
|
||||
public final static class InsanityType {
|
||||
private final String label;
|
||||
private InsanityType(final String label) {
|
||||
this.label = label;
|
||||
}
|
||||
@Override
|
||||
public String toString() { return label; }
|
||||
|
||||
/**
|
||||
* Indicates an overlap in cache usage on a given field
|
||||
* in sub/super readers.
|
||||
*/
|
||||
public final static InsanityType SUBREADER
|
||||
= new InsanityType("SUBREADER");
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Indicates entries have the same reader+fieldname but
|
||||
* different cached values. This can happen if different datatypes,
|
||||
* or parsers are used -- and while it's not necessarily a bug
|
||||
* it's typically an indication of a possible problem.
|
||||
* </p>
|
||||
* <p>
|
||||
* <b>NOTE:</b> Only the reader, fieldname, and cached value are actually
|
||||
* tested -- if two cache entries have different parsers or datatypes but
|
||||
* the cached values are the same Object (== not just equal()) this method
|
||||
* does not consider that a red flag. This allows for subtle variations
|
||||
* in the way a Parser is specified (null vs DEFAULT_LONG_PARSER, etc...)
|
||||
* </p>
|
||||
*/
|
||||
public final static InsanityType VALUEMISMATCH
|
||||
= new InsanityType("VALUEMISMATCH");
|
||||
|
||||
/**
|
||||
* Indicates an expected bit of "insanity". This may be useful for
|
||||
* clients that wish to preserve/log information about insane usage
|
||||
* but indicate that it was expected.
|
||||
*/
|
||||
public final static InsanityType EXPECTED
|
||||
= new InsanityType("EXPECTED");
|
||||
}
|
||||
|
||||
|
||||
}
|
|
@ -227,6 +227,15 @@ public class UninvertingReader extends FilterLeafReader {
|
|||
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
|
||||
return new UninvertingDirectoryReader(in, mapping);
|
||||
}
|
||||
|
||||
// NOTE: delegating the cache helpers is wrong since this wrapper alters the
|
||||
// content of the reader, it is only fine to do that because Solr ALWAYS
|
||||
// consumes index readers through this wrapper
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return in.getReaderCacheHelper();
|
||||
}
|
||||
}
|
||||
|
||||
final Map<String,Type> mapping;
|
||||
|
@ -391,14 +400,18 @@ public class UninvertingReader extends FilterLeafReader {
|
|||
return mapping.get(field);
|
||||
}
|
||||
|
||||
// NOTE: delegating the cache helpers is wrong since this wrapper alters the
|
||||
// content of the reader, it is only fine to do that because Solr ALWAYS
|
||||
// consumes index readers through this wrapper
|
||||
|
||||
@Override
|
||||
public Object getCoreCacheKey() {
|
||||
return in.getCoreCacheKey();
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return in.getCoreCacheHelper();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getCombinedCoreAndDeletesKey() {
|
||||
return in.getCombinedCoreAndDeletesKey();
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return in.getReaderCacheHelper();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -289,6 +289,16 @@ public class SolrIndexSplitter {
|
|||
public Bits getLiveDocs() {
|
||||
return liveDocs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return in.getCoreCacheHelper();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -145,7 +145,7 @@ public class TestNRTOpen extends SolrTestCaseJ4 {
|
|||
try {
|
||||
DirectoryReader ir = searcher.get().getRawReader();
|
||||
for (LeafReaderContext context : ir.leaves()) {
|
||||
set.add(context.reader().getCoreCacheKey());
|
||||
set.add(context.reader().getCoreCacheHelper().getKey());
|
||||
}
|
||||
} finally {
|
||||
searcher.decref();
|
||||
|
|
|
@ -18,15 +18,20 @@ package org.apache.solr.index;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.SortedDocValuesField;
|
||||
import org.apache.lucene.document.SortedSetDocValuesField;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.MultiDocValues.MultiSortedDocValues;
|
||||
import org.apache.lucene.index.MultiDocValues.MultiSortedSetDocValues;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
|
@ -48,21 +53,16 @@ public class TestSlowCompositeReaderWrapper extends LuceneTestCase {
|
|||
final LeafReader leafReader = SlowCompositeReaderWrapper.wrap(reader);
|
||||
|
||||
final int numListeners = TestUtil.nextInt(random(), 1, 10);
|
||||
final List<LeafReader.CoreClosedListener> listeners = new ArrayList<>();
|
||||
final List<IndexReader.ClosedListener> listeners = new ArrayList<>();
|
||||
AtomicInteger counter = new AtomicInteger(numListeners);
|
||||
|
||||
for (int i = 0; i < numListeners; ++i) {
|
||||
CountCoreListener listener = new CountCoreListener(counter, leafReader.getCoreCacheKey());
|
||||
CountCoreListener listener = new CountCoreListener(counter, leafReader.getCoreCacheHelper().getKey());
|
||||
listeners.add(listener);
|
||||
leafReader.addCoreClosedListener(listener);
|
||||
leafReader.getCoreCacheHelper().addClosedListener(listener);
|
||||
}
|
||||
for (int i = 0; i < 100; ++i) {
|
||||
leafReader.addCoreClosedListener(listeners.get(random().nextInt(listeners.size())));
|
||||
}
|
||||
final int removed = random().nextInt(numListeners);
|
||||
Collections.shuffle(listeners, random());
|
||||
for (int i = 0; i < removed; ++i) {
|
||||
leafReader.removeCoreClosedListener(listeners.get(i));
|
||||
leafReader.getCoreCacheHelper().addClosedListener(listeners.get(random().nextInt(listeners.size())));
|
||||
}
|
||||
assertEquals(numListeners, counter.get());
|
||||
// make sure listeners are registered on the wrapped reader and that closing any of them has the same effect
|
||||
|
@ -71,11 +71,11 @@ public class TestSlowCompositeReaderWrapper extends LuceneTestCase {
|
|||
} else {
|
||||
leafReader.close();
|
||||
}
|
||||
assertEquals(removed, counter.get());
|
||||
assertEquals(0, counter.get());
|
||||
w.w.getDirectory().close();
|
||||
}
|
||||
|
||||
private static final class CountCoreListener implements LeafReader.CoreClosedListener {
|
||||
private static final class CountCoreListener implements IndexReader.ClosedListener {
|
||||
|
||||
private final AtomicInteger count;
|
||||
private final Object coreCacheKey;
|
||||
|
@ -86,10 +86,37 @@ public class TestSlowCompositeReaderWrapper extends LuceneTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void onClose(Object coreCacheKey) {
|
||||
public void onClose(IndexReader.CacheKey coreCacheKey) {
|
||||
assertSame(this.coreCacheKey, coreCacheKey);
|
||||
count.decrementAndGet();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public void testOrdMapsAreCached() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir);
|
||||
Document doc = new Document();
|
||||
doc.add(new SortedDocValuesField("sorted", new BytesRef("a")));
|
||||
doc.add(new SortedSetDocValuesField("sorted_set", new BytesRef("b")));
|
||||
doc.add(new SortedSetDocValuesField("sorted_set", new BytesRef("c")));
|
||||
w.addDocument(doc);
|
||||
w.getReader().close();
|
||||
doc = new Document();
|
||||
doc.add(new SortedDocValuesField("sorted", new BytesRef("b")));
|
||||
doc.add(new SortedSetDocValuesField("sorted_set", new BytesRef("c")));
|
||||
doc.add(new SortedSetDocValuesField("sorted_set", new BytesRef("d")));
|
||||
w.addDocument(doc);
|
||||
IndexReader reader = w.getReader();
|
||||
assertTrue(reader.leaves().size() > 1);
|
||||
SlowCompositeReaderWrapper slowWrapper = (SlowCompositeReaderWrapper) SlowCompositeReaderWrapper.wrap(reader);
|
||||
assertEquals(0, slowWrapper.cachedOrdMaps.size());
|
||||
assertEquals(MultiSortedDocValues.class, slowWrapper.getSortedDocValues("sorted").getClass());
|
||||
assertEquals(1, slowWrapper.cachedOrdMaps.size());
|
||||
assertEquals(MultiSortedSetDocValues.class, slowWrapper.getSortedSetDocValues("sorted_set").getClass());
|
||||
assertEquals(2, slowWrapper.cachedOrdMaps.size());
|
||||
reader.close();
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -387,16 +387,6 @@ public class TestDocSet extends LuceneTestCase {
|
|||
return maxDoc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addCoreClosedListener(CoreClosedListener listener) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeCoreClosedListener(CoreClosedListener listener) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldInfos getFieldInfos() {
|
||||
return new FieldInfos(new FieldInfo[0]);
|
||||
|
@ -468,6 +458,16 @@ public class TestDocSet extends LuceneTestCase {
|
|||
public Sort getIndexSort() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return null;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
|
|
@ -157,7 +157,7 @@ public class TestSolr4Spatial2 extends SolrTestCaseJ4 {
|
|||
|
||||
|
||||
protected Object getFirstLeafReaderKey() {
|
||||
return getSearcher().getRawReader().leaves().get(0).reader().getCoreCacheKey();
|
||||
return getSearcher().getRawReader().leaves().get(0).reader().getCoreCacheHelper().getKey();
|
||||
}
|
||||
|
||||
@Test// SOLR-8541
|
||||
|
|
|
@ -218,7 +218,7 @@ public class TestDocTermOrds extends LuceneTestCase {
|
|||
TestUtil.checkReader(slowR);
|
||||
verify(slowR, idToOrds, termsArray, null);
|
||||
|
||||
FieldCache.DEFAULT.purgeByCacheKey(slowR.getCoreCacheKey());
|
||||
FieldCache.DEFAULT.purgeByCacheKey(slowR.getCoreCacheHelper().getKey());
|
||||
|
||||
r.close();
|
||||
dir.close();
|
||||
|
@ -338,7 +338,7 @@ public class TestDocTermOrds extends LuceneTestCase {
|
|||
verify(slowR, idToOrdsPrefix, termsArray, prefixRef);
|
||||
}
|
||||
|
||||
FieldCache.DEFAULT.purgeByCacheKey(slowR.getCoreCacheKey());
|
||||
FieldCache.DEFAULT.purgeByCacheKey(slowR.getCoreCacheHelper().getKey());
|
||||
|
||||
r.close();
|
||||
dir.close();
|
||||
|
|
|
@ -267,7 +267,7 @@ public class TestFieldCache extends LuceneTestCase {
|
|||
termOrds = cache.getDocTermOrds(reader, "bogusfield", null);
|
||||
assertTrue(termOrds.getValueCount() == 0);
|
||||
|
||||
FieldCache.DEFAULT.purgeByCacheKey(reader.getCoreCacheKey());
|
||||
FieldCache.DEFAULT.purgeByCacheKey(reader.getCoreCacheHelper().getKey());
|
||||
}
|
||||
|
||||
public void testEmptyIndex() throws Exception {
|
||||
|
@ -279,7 +279,7 @@ public class TestFieldCache extends LuceneTestCase {
|
|||
TestUtil.checkReader(reader);
|
||||
FieldCache.DEFAULT.getTerms(reader, "foobar");
|
||||
FieldCache.DEFAULT.getTermsIndex(reader, "foobar");
|
||||
FieldCache.DEFAULT.purgeByCacheKey(reader.getCoreCacheKey());
|
||||
FieldCache.DEFAULT.purgeByCacheKey(reader.getCoreCacheHelper().getKey());
|
||||
r.close();
|
||||
dir.close();
|
||||
}
|
||||
|
|
|
@ -1,164 +0,0 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.solr.uninverting;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.MultiReader;
|
||||
import org.apache.lucene.legacy.LegacyDoubleField;
|
||||
import org.apache.lucene.legacy.LegacyFloatField;
|
||||
import org.apache.lucene.legacy.LegacyIntField;
|
||||
import org.apache.lucene.legacy.LegacyLongField;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.solr.index.SlowCompositeReaderWrapper;
|
||||
import org.apache.solr.uninverting.FieldCacheSanityChecker.Insanity;
|
||||
import org.apache.solr.uninverting.FieldCacheSanityChecker.InsanityType;
|
||||
|
||||
public class TestFieldCacheSanityChecker extends LuceneTestCase {
|
||||
|
||||
protected LeafReader readerA;
|
||||
protected LeafReader readerB;
|
||||
protected LeafReader readerX;
|
||||
protected LeafReader readerAclone;
|
||||
protected Directory dirA, dirB;
|
||||
private static final int NUM_DOCS = 1000;
|
||||
|
||||
@Override
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
dirA = newDirectory();
|
||||
dirB = newDirectory();
|
||||
|
||||
IndexWriter wA = new IndexWriter(dirA, newIndexWriterConfig(new MockAnalyzer(random())));
|
||||
IndexWriter wB = new IndexWriter(dirB, newIndexWriterConfig(new MockAnalyzer(random())));
|
||||
|
||||
long theLong = Long.MAX_VALUE;
|
||||
double theDouble = Double.MAX_VALUE;
|
||||
int theInt = Integer.MAX_VALUE;
|
||||
float theFloat = Float.MAX_VALUE;
|
||||
for (int i = 0; i < NUM_DOCS; i++){
|
||||
Document doc = new Document();
|
||||
doc.add(new LegacyLongField("theLong", theLong--, Field.Store.NO));
|
||||
doc.add(new LegacyDoubleField("theDouble", theDouble--, Field.Store.NO));
|
||||
doc.add(new LegacyIntField("theInt", theInt--, Field.Store.NO));
|
||||
doc.add(new LegacyFloatField("theFloat", theFloat--, Field.Store.NO));
|
||||
if (0 == i % 3) {
|
||||
wA.addDocument(doc);
|
||||
} else {
|
||||
wB.addDocument(doc);
|
||||
}
|
||||
}
|
||||
wA.close();
|
||||
wB.close();
|
||||
DirectoryReader rA = DirectoryReader.open(dirA);
|
||||
readerA = SlowCompositeReaderWrapper.wrap(rA);
|
||||
readerAclone = SlowCompositeReaderWrapper.wrap(rA);
|
||||
readerA = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(dirA));
|
||||
readerB = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(dirB));
|
||||
readerX = SlowCompositeReaderWrapper.wrap(new MultiReader(readerA, readerB));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void tearDown() throws Exception {
|
||||
readerA.close();
|
||||
readerAclone.close();
|
||||
readerB.close();
|
||||
readerX.close();
|
||||
dirA.close();
|
||||
dirB.close();
|
||||
super.tearDown();
|
||||
}
|
||||
|
||||
public void testSanity() throws IOException {
|
||||
FieldCache cache = FieldCache.DEFAULT;
|
||||
cache.purgeAllCaches();
|
||||
|
||||
cache.getNumerics(readerA, "theDouble", FieldCache.LEGACY_DOUBLE_PARSER);
|
||||
cache.getNumerics(readerAclone, "theDouble", FieldCache.LEGACY_DOUBLE_PARSER);
|
||||
cache.getNumerics(readerB, "theDouble", FieldCache.LEGACY_DOUBLE_PARSER);
|
||||
|
||||
cache.getNumerics(readerX, "theInt", FieldCache.LEGACY_INT_PARSER);
|
||||
|
||||
// // //
|
||||
|
||||
Insanity[] insanity =
|
||||
FieldCacheSanityChecker.checkSanity(cache.getCacheEntries());
|
||||
|
||||
if (0 < insanity.length)
|
||||
dumpArray(getTestClass().getName() + "#" + getTestName()
|
||||
+ " INSANITY", insanity, System.err);
|
||||
|
||||
assertEquals("shouldn't be any cache insanity", 0, insanity.length);
|
||||
cache.purgeAllCaches();
|
||||
}
|
||||
|
||||
public void testInsanity1() throws IOException {
|
||||
FieldCache cache = FieldCache.DEFAULT;
|
||||
cache.purgeAllCaches();
|
||||
|
||||
cache.getNumerics(readerX, "theInt", FieldCache.LEGACY_INT_PARSER);
|
||||
cache.getTerms(readerX, "theInt");
|
||||
|
||||
// // //
|
||||
|
||||
Insanity[] insanity =
|
||||
FieldCacheSanityChecker.checkSanity(cache.getCacheEntries());
|
||||
|
||||
assertEquals("wrong number of cache errors", 1, insanity.length);
|
||||
assertEquals("wrong type of cache error",
|
||||
InsanityType.VALUEMISMATCH,
|
||||
insanity[0].getType());
|
||||
assertEquals("wrong number of entries in cache error", 2,
|
||||
insanity[0].getCacheEntries().length);
|
||||
|
||||
// we expect bad things, don't let tearDown complain about them
|
||||
cache.purgeAllCaches();
|
||||
}
|
||||
|
||||
public void testInsanity2() throws IOException {
|
||||
FieldCache cache = FieldCache.DEFAULT;
|
||||
cache.purgeAllCaches();
|
||||
|
||||
cache.getTerms(readerA, "theInt");
|
||||
cache.getTerms(readerB, "theInt");
|
||||
cache.getTerms(readerX, "theInt");
|
||||
|
||||
|
||||
// // //
|
||||
|
||||
Insanity[] insanity =
|
||||
FieldCacheSanityChecker.checkSanity(cache.getCacheEntries());
|
||||
|
||||
assertEquals("wrong number of cache errors", 1, insanity.length);
|
||||
assertEquals("wrong type of cache error",
|
||||
InsanityType.SUBREADER,
|
||||
insanity[0].getType());
|
||||
assertEquals("wrong number of entries in cache error", 3,
|
||||
insanity[0].getCacheEntries().length);
|
||||
|
||||
// we expect bad things, don't let tearDown complain about them
|
||||
cache.purgeAllCaches();
|
||||
}
|
||||
|
||||
}
|
|
@ -32,26 +32,20 @@ import org.apache.lucene.index.IndexWriterConfig;
|
|||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.NumericDocValues;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.legacy.LegacyDoubleField;
|
||||
import org.apache.lucene.legacy.LegacyFloatField;
|
||||
import org.apache.lucene.legacy.LegacyIntField;
|
||||
import org.apache.lucene.legacy.LegacyLongField;
|
||||
import org.apache.lucene.legacy.LegacyNumericUtils;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
import org.apache.solr.index.SlowCompositeReaderWrapper;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.CyclicBarrier;
|
||||
|
@ -107,31 +101,6 @@ public class TestLegacyFieldCache extends LuceneTestCase {
|
|||
directory = null;
|
||||
}
|
||||
|
||||
public void testInfoStream() throws Exception {
|
||||
try {
|
||||
FieldCache cache = FieldCache.DEFAULT;
|
||||
ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
|
||||
cache.setInfoStream(new PrintStream(bos, false, IOUtils.UTF_8));
|
||||
cache.getNumerics(reader, "theDouble", FieldCache.LEGACY_DOUBLE_PARSER);
|
||||
cache.getNumerics(reader, "theDouble", new FieldCache.Parser() {
|
||||
@Override
|
||||
public TermsEnum termsEnum(Terms terms) throws IOException {
|
||||
return LegacyNumericUtils.filterPrefixCodedLongs(terms.iterator());
|
||||
}
|
||||
@Override
|
||||
public long parseValue(BytesRef term) {
|
||||
int val = (int) LegacyNumericUtils.prefixCodedToLong(term);
|
||||
if (val<0) val ^= 0x7fffffff;
|
||||
return val;
|
||||
}
|
||||
});
|
||||
assertTrue(bos.toString(IOUtils.UTF_8).indexOf("WARNING") != -1);
|
||||
} finally {
|
||||
FieldCache.DEFAULT.setInfoStream(null);
|
||||
FieldCache.DEFAULT.purgeAllCaches();
|
||||
}
|
||||
}
|
||||
|
||||
public void test() throws IOException {
|
||||
FieldCache cache = FieldCache.DEFAULT;
|
||||
NumericDocValues doubles = cache.getNumerics(reader, "theDouble", FieldCache.LEGACY_DOUBLE_PARSER);
|
||||
|
@ -174,7 +143,7 @@ public class TestLegacyFieldCache extends LuceneTestCase {
|
|||
assertEquals(i%2 == 0, docsWithField.get(i));
|
||||
}
|
||||
|
||||
FieldCache.DEFAULT.purgeByCacheKey(reader.getCoreCacheKey());
|
||||
FieldCache.DEFAULT.purgeByCacheKey(reader.getCoreCacheHelper().getKey());
|
||||
}
|
||||
|
||||
public void testEmptyIndex() throws Exception {
|
||||
|
@ -186,7 +155,7 @@ public class TestLegacyFieldCache extends LuceneTestCase {
|
|||
TestUtil.checkReader(reader);
|
||||
FieldCache.DEFAULT.getTerms(reader, "foobar");
|
||||
FieldCache.DEFAULT.getTermsIndex(reader, "foobar");
|
||||
FieldCache.DEFAULT.purgeByCacheKey(reader.getCoreCacheKey());
|
||||
FieldCache.DEFAULT.purgeByCacheKey(reader.getCoreCacheHelper().getKey());
|
||||
r.close();
|
||||
dir.close();
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue