mirror of https://github.com/apache/lucene.git
LUCENE-5254: don't hold ref to original SR from SCR, to avoid bounded leak of things like live docs bitset
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1529135 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
16ce59c34a
commit
1ea2fade59
|
@ -94,6 +94,11 @@ Bug Fixes
|
|||
its state, which could result in exceptions being thrown, as well as
|
||||
incorrect ordinals returned from getParent. (Shai Erera)
|
||||
|
||||
* LUCENE-5254: Fixed bounded memory leak, where objects like live
|
||||
docs bitset were not freed from an starting reader after reopening
|
||||
to a new reader and closing the original one. (Shai Erera, Mike
|
||||
McCandless)
|
||||
|
||||
API Changes:
|
||||
|
||||
* LUCENE-5222: Add SortField.needsScores(). Previously it was not possible
|
||||
|
|
|
@ -53,7 +53,7 @@ final class SegmentCoreReaders {
|
|||
final FieldsProducer fields;
|
||||
final DocValuesProducer normsProducer;
|
||||
|
||||
private final SegmentReader owner;
|
||||
private final Object ownerCoreCacheKey;
|
||||
|
||||
final StoredFieldsReader fieldsReaderOrig;
|
||||
final TermVectorsReader termVectorsReaderOrig;
|
||||
|
@ -88,7 +88,12 @@ final class SegmentCoreReaders {
|
|||
Collections.synchronizedSet(new LinkedHashSet<CoreClosedListener>());
|
||||
|
||||
SegmentCoreReaders(SegmentReader owner, Directory dir, SegmentInfoPerCommit si, IOContext context) throws IOException {
|
||||
|
||||
|
||||
// SegmentReader uses us as the coreCacheKey; we cannot
|
||||
// call owner.getCoreCacheKey() because that will return
|
||||
// null!:
|
||||
this.ownerCoreCacheKey = this;
|
||||
|
||||
final Codec codec = si.info.getCodec();
|
||||
final Directory cfsDir; // confusing name: if (cfs) its the cfsdir, otherwise its the segment's directory.
|
||||
|
||||
|
@ -134,12 +139,6 @@ final class SegmentCoreReaders {
|
|||
decRef();
|
||||
}
|
||||
}
|
||||
|
||||
// Must assign this at the end -- if we hit an
|
||||
// exception above core, we don't want to attempt to
|
||||
// purge the FieldCache (will hit NPE because core is
|
||||
// not assigned yet).
|
||||
this.owner = owner;
|
||||
}
|
||||
|
||||
int getRefCount() {
|
||||
|
@ -176,7 +175,7 @@ final class SegmentCoreReaders {
|
|||
private void notifyCoreClosedListeners() {
|
||||
synchronized(coreClosedListeners) {
|
||||
for (CoreClosedListener listener : coreClosedListeners) {
|
||||
listener.onClose(owner);
|
||||
listener.onClose(ownerCoreCacheKey);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -196,9 +195,4 @@ final class SegmentCoreReaders {
|
|||
((fieldsReaderOrig!=null)? fieldsReaderOrig.ramBytesUsed() : 0) +
|
||||
((termVectorsReaderOrig!=null) ? termVectorsReaderOrig.ramBytesUsed() : 0);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "SegmentCoreReader(owner=" + owner + ")";
|
||||
}
|
||||
}
|
||||
|
|
|
@ -407,6 +407,8 @@ public final class SegmentReader extends AtomicReader {
|
|||
// same entry in the FieldCache. See LUCENE-1579.
|
||||
@Override
|
||||
public Object getCoreCacheKey() {
|
||||
// NOTE: if this every changes, be sure to fix
|
||||
// SegmentCoreReader's ownerCoreCacheKey to match!
|
||||
return core;
|
||||
}
|
||||
|
||||
|
@ -576,9 +578,9 @@ public final class SegmentReader extends AtomicReader {
|
|||
* @lucene.experimental
|
||||
*/
|
||||
public static interface CoreClosedListener {
|
||||
/** Invoked when the shared core of the provided {@link
|
||||
/** Invoked when the shared core of the original {@code
|
||||
* SegmentReader} has closed. */
|
||||
public void onClose(SegmentReader owner);
|
||||
public void onClose(Object ownerCoreCacheKey);
|
||||
}
|
||||
|
||||
/** Expert: adds a CoreClosedListener to this reader's shared core */
|
||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.lucene.document.NumericDocValuesField;
|
|||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.BinaryDocValues;
|
||||
import org.apache.lucene.index.DocTermOrds;
|
||||
import org.apache.lucene.index.IndexReader; // javadocs
|
||||
import org.apache.lucene.index.SortedDocValues;
|
||||
import org.apache.lucene.index.SortedSetDocValues;
|
||||
import org.apache.lucene.index.Terms;
|
||||
|
@ -260,7 +261,7 @@ public interface FieldCache {
|
|||
*
|
||||
* @see #getInts(AtomicReader, String, IntParser, boolean)
|
||||
*/
|
||||
public Ints getInts (AtomicReader reader, String field, boolean setDocsWithField) throws IOException;
|
||||
public Ints getInts(AtomicReader reader, String field, boolean setDocsWithField) throws IOException;
|
||||
|
||||
/**
|
||||
* Returns an {@link Ints} over the values found in documents in the given
|
||||
|
@ -286,7 +287,7 @@ public interface FieldCache {
|
|||
* @throws IOException
|
||||
* If any error occurs.
|
||||
*/
|
||||
public Ints getInts (AtomicReader reader, String field, IntParser parser, boolean setDocsWithField) throws IOException;
|
||||
public Ints getInts(AtomicReader reader, String field, IntParser parser, boolean setDocsWithField) throws IOException;
|
||||
|
||||
/**
|
||||
* Returns a {@link Floats} over the values found in documents in the given
|
||||
|
@ -294,7 +295,7 @@ public interface FieldCache {
|
|||
*
|
||||
* @see #getFloats(AtomicReader, String, FloatParser, boolean)
|
||||
*/
|
||||
public Floats getFloats (AtomicReader reader, String field, boolean setDocsWithField) throws IOException;
|
||||
public Floats getFloats(AtomicReader reader, String field, boolean setDocsWithField) throws IOException;
|
||||
|
||||
/**
|
||||
* Returns a {@link Floats} over the values found in documents in the given
|
||||
|
@ -320,7 +321,7 @@ public interface FieldCache {
|
|||
* @throws IOException
|
||||
* If any error occurs.
|
||||
*/
|
||||
public Floats getFloats (AtomicReader reader, String field, FloatParser parser, boolean setDocsWithField) throws IOException;
|
||||
public Floats getFloats(AtomicReader reader, String field, FloatParser parser, boolean setDocsWithField) throws IOException;
|
||||
|
||||
/**
|
||||
* Returns a {@link Longs} over the values found in documents in the given
|
||||
|
@ -401,14 +402,14 @@ public interface FieldCache {
|
|||
* @return The values in the given field for each document.
|
||||
* @throws IOException If any error occurs.
|
||||
*/
|
||||
public BinaryDocValues getTerms (AtomicReader reader, String field, boolean setDocsWithField) throws IOException;
|
||||
public BinaryDocValues getTerms(AtomicReader reader, String field, boolean setDocsWithField) throws IOException;
|
||||
|
||||
/** Expert: just like {@link #getTerms(AtomicReader,String,boolean)},
|
||||
* but you can specify whether more RAM should be consumed in exchange for
|
||||
* faster lookups (default is "true"). Note that the
|
||||
* first call for a given reader and field "wins",
|
||||
* subsequent calls will share the same cache entry. */
|
||||
public BinaryDocValues getTerms (AtomicReader reader, String field, boolean setDocsWithField, float acceptableOverheadRatio) throws IOException;
|
||||
public BinaryDocValues getTerms(AtomicReader reader, String field, boolean setDocsWithField, float acceptableOverheadRatio) throws IOException;
|
||||
|
||||
/** Checks the internal cache for an appropriate entry, and if none
|
||||
* is found, reads the term values in <code>field</code>
|
||||
|
@ -420,7 +421,7 @@ public interface FieldCache {
|
|||
* @return The values in the given field for each document.
|
||||
* @throws IOException If any error occurs.
|
||||
*/
|
||||
public SortedDocValues getTermsIndex (AtomicReader reader, String field) throws IOException;
|
||||
public SortedDocValues getTermsIndex(AtomicReader reader, String field) throws IOException;
|
||||
|
||||
/** Expert: just like {@link
|
||||
* #getTermsIndex(AtomicReader,String)}, but you can specify
|
||||
|
@ -428,7 +429,7 @@ public interface FieldCache {
|
|||
* faster lookups (default is "true"). Note that the
|
||||
* first call for a given reader and field "wins",
|
||||
* subsequent calls will share the same cache entry. */
|
||||
public SortedDocValues getTermsIndex (AtomicReader reader, String field, float acceptableOverheadRatio) throws IOException;
|
||||
public SortedDocValues getTermsIndex(AtomicReader reader, String field, float acceptableOverheadRatio) throws IOException;
|
||||
|
||||
/**
|
||||
* Checks the internal cache for an appropriate entry, and if none is found, reads the term values
|
||||
|
@ -533,7 +534,7 @@ public interface FieldCache {
|
|||
* </p>
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public abstract CacheEntry[] getCacheEntries();
|
||||
public CacheEntry[] getCacheEntries();
|
||||
|
||||
/**
|
||||
* <p>
|
||||
|
@ -546,16 +547,17 @@ public interface FieldCache {
|
|||
* </p>
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public abstract void purgeAllCaches();
|
||||
public void purgeAllCaches();
|
||||
|
||||
/**
|
||||
* Expert: drops all cache entries associated with this
|
||||
* reader. NOTE: this reader 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.
|
||||
* reader {@link IndexReader#getCoreCacheKey}. 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 abstract void purge(AtomicReader r);
|
||||
public void purgeByCacheKey(Object coreCacheKey);
|
||||
|
||||
/**
|
||||
* If non-null, FieldCacheImpl will warn whenever
|
||||
|
|
|
@ -78,9 +78,9 @@ class FieldCacheImpl implements FieldCache {
|
|||
}
|
||||
|
||||
@Override
|
||||
public synchronized void purge(AtomicReader r) {
|
||||
public synchronized void purgeByCacheKey(Object coreCacheKey) {
|
||||
for(Cache c : caches.values()) {
|
||||
c.purge(r);
|
||||
c.purgeByCacheKey(coreCacheKey);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -110,8 +110,8 @@ class FieldCacheImpl implements FieldCache {
|
|||
// per-segment fieldcaches don't purge until the shared core closes.
|
||||
final SegmentReader.CoreClosedListener purgeCore = new SegmentReader.CoreClosedListener() {
|
||||
@Override
|
||||
public void onClose(SegmentReader owner) {
|
||||
FieldCacheImpl.this.purge(owner);
|
||||
public void onClose(Object ownerCoreCacheKey) {
|
||||
FieldCacheImpl.this.purgeByCacheKey(ownerCoreCacheKey);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -120,7 +120,7 @@ class FieldCacheImpl implements FieldCache {
|
|||
@Override
|
||||
public void onClose(IndexReader owner) {
|
||||
assert owner instanceof AtomicReader;
|
||||
FieldCacheImpl.this.purge((AtomicReader) owner);
|
||||
FieldCacheImpl.this.purgeByCacheKey(((AtomicReader) owner).getCoreCacheKey());
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -155,10 +155,9 @@ class FieldCacheImpl implements FieldCache {
|
|||
throws IOException;
|
||||
|
||||
/** Remove this reader from the cache, if present. */
|
||||
public void purge(AtomicReader r) {
|
||||
Object readerKey = r.getCoreCacheKey();
|
||||
public void purgeByCacheKey(Object coreCacheKey) {
|
||||
synchronized(readerCache) {
|
||||
readerCache.remove(readerKey);
|
||||
readerCache.remove(coreCacheKey);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -172,7 +172,7 @@ public class TestDocTermOrds extends LuceneTestCase {
|
|||
AtomicReader slowR = SlowCompositeReaderWrapper.wrap(r);
|
||||
verify(slowR, idToOrds, termsArray, null);
|
||||
|
||||
FieldCache.DEFAULT.purge(slowR);
|
||||
FieldCache.DEFAULT.purgeByCacheKey(slowR.getCoreCacheKey());
|
||||
|
||||
r.close();
|
||||
dir.close();
|
||||
|
@ -291,7 +291,7 @@ public class TestDocTermOrds extends LuceneTestCase {
|
|||
verify(slowR, idToOrdsPrefix, termsArray, prefixRef);
|
||||
}
|
||||
|
||||
FieldCache.DEFAULT.purge(slowR);
|
||||
FieldCache.DEFAULT.purgeByCacheKey(slowR.getCoreCacheKey());
|
||||
|
||||
r.close();
|
||||
dir.close();
|
||||
|
|
|
@ -297,7 +297,7 @@ public class TestFieldCache extends LuceneTestCase {
|
|||
termOrds = cache.getDocTermOrds(reader, "bogusfield");
|
||||
assertTrue(termOrds.getValueCount() == 0);
|
||||
|
||||
FieldCache.DEFAULT.purge(reader);
|
||||
FieldCache.DEFAULT.purgeByCacheKey(reader.getCoreCacheKey());
|
||||
}
|
||||
|
||||
public void testEmptyIndex() throws Exception {
|
||||
|
@ -308,7 +308,7 @@ public class TestFieldCache extends LuceneTestCase {
|
|||
AtomicReader reader = SlowCompositeReaderWrapper.wrap(r);
|
||||
FieldCache.DEFAULT.getTerms(reader, "foobar", true);
|
||||
FieldCache.DEFAULT.getTermsIndex(reader, "foobar");
|
||||
FieldCache.DEFAULT.purge(reader);
|
||||
FieldCache.DEFAULT.purgeByCacheKey(reader.getCoreCacheKey());
|
||||
r.close();
|
||||
dir.close();
|
||||
}
|
||||
|
|
|
@ -132,7 +132,7 @@ public class QueryUtils {
|
|||
|
||||
public static void purgeFieldCache(IndexReader r) throws IOException {
|
||||
// this is just a hack, to get an atomic reader that contains all subreaders for insanity checks
|
||||
FieldCache.DEFAULT.purge(SlowCompositeReaderWrapper.wrap(r));
|
||||
FieldCache.DEFAULT.purgeByCacheKey(SlowCompositeReaderWrapper.wrap(r).getCoreCacheKey());
|
||||
}
|
||||
|
||||
/** This is a MultiReader that can be used for randomly wrapping other readers
|
||||
|
|
Loading…
Reference in New Issue