mirror of https://github.com/apache/lucene.git
LUCENE-3644: fix problems with IR's readerFinishedListener
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1213910 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
0b734cfe20
commit
1a4c0ff254
|
@ -686,6 +686,14 @@ Changes in backwards compatibility policy
|
|||
even return your own subclass of IndexSearcher. The SearcherWarmer and
|
||||
ExecutorService parameters on these classes were removed, as they are
|
||||
subsumed by SearcherFactory. (Shai Erera, Mike McCandless, Robert Muir)
|
||||
|
||||
* LUCENE-3644: The expert ReaderFinishedListener api suffered problems (propagated
|
||||
down to subreaders, but was not called on SegmentReaders, unless they were
|
||||
the owner of the reader core, and other ambiguities). The API is revised:
|
||||
You can set ReaderClosedListeners on any IndexReader, and onClose is called
|
||||
when that reader is closed. SegmentReader has CoreClosedListeners that you
|
||||
can register to know when a shared reader core is closed.
|
||||
(Uwe Schindler, Mike McCandless, Robert Muir)
|
||||
|
||||
Security fixes
|
||||
|
||||
|
|
|
@ -24,7 +24,6 @@ import java.util.Collection;
|
|||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -750,7 +749,6 @@ public class MemoryIndex {
|
|||
|
||||
private MemoryIndexReader() {
|
||||
super(); // avoid as much superclass baggage as possible
|
||||
readerFinishedListeners = Collections.synchronizedSet(new HashSet<ReaderFinishedListener>());
|
||||
}
|
||||
|
||||
private Info getInfo(String fieldName) {
|
||||
|
|
|
@ -25,12 +25,10 @@ import java.util.Collections;
|
|||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.MapBackedSet;
|
||||
|
||||
/**
|
||||
* An IndexReader which reads indexes with multiple segments.
|
||||
|
@ -43,15 +41,12 @@ final class DirectoryReader extends BaseMultiReader<SegmentReader> {
|
|||
private final boolean applyAllDeletes;
|
||||
|
||||
DirectoryReader(SegmentReader[] readers, Directory directory, IndexWriter writer,
|
||||
SegmentInfos sis, int termInfosIndexDivisor, boolean applyAllDeletes,
|
||||
Collection<ReaderFinishedListener> readerFinishedListeners
|
||||
) throws IOException {
|
||||
SegmentInfos sis, int termInfosIndexDivisor, boolean applyAllDeletes) throws IOException {
|
||||
super(readers);
|
||||
this.directory = directory;
|
||||
this.writer = writer;
|
||||
this.segmentInfos = sis;
|
||||
this.termInfosIndexDivisor = termInfosIndexDivisor;
|
||||
this.readerFinishedListeners = readerFinishedListeners;
|
||||
this.applyAllDeletes = applyAllDeletes;
|
||||
}
|
||||
|
||||
|
@ -60,8 +55,6 @@ final class DirectoryReader extends BaseMultiReader<SegmentReader> {
|
|||
return (IndexReader) new SegmentInfos.FindSegmentsFile(directory) {
|
||||
@Override
|
||||
protected Object doBody(String segmentFileName) throws CorruptIndexException, IOException {
|
||||
final Collection<ReaderFinishedListener> readerFinishedListeners =
|
||||
new MapBackedSet<ReaderFinishedListener>(new ConcurrentHashMap<ReaderFinishedListener,Boolean>());
|
||||
SegmentInfos sis = new SegmentInfos();
|
||||
sis.read(directory, segmentFileName);
|
||||
final SegmentReader[] readers = new SegmentReader[sis.size()];
|
||||
|
@ -70,7 +63,6 @@ final class DirectoryReader extends BaseMultiReader<SegmentReader> {
|
|||
boolean success = false;
|
||||
try {
|
||||
readers[i] = SegmentReader.get(sis.info(i), termInfosIndexDivisor, IOContext.READ);
|
||||
readers[i].readerFinishedListeners = readerFinishedListeners;
|
||||
success = true;
|
||||
} catch(IOException ex) {
|
||||
prior = ex;
|
||||
|
@ -79,8 +71,7 @@ final class DirectoryReader extends BaseMultiReader<SegmentReader> {
|
|||
IOUtils.closeWhileHandlingException(prior, readers);
|
||||
}
|
||||
}
|
||||
return new DirectoryReader(readers, directory, null, sis, termInfosIndexDivisor,
|
||||
false, readerFinishedListeners);
|
||||
return new DirectoryReader(readers, directory, null, sis, termInfosIndexDivisor, false);
|
||||
}
|
||||
}.run(commit);
|
||||
}
|
||||
|
@ -105,7 +96,6 @@ final class DirectoryReader extends BaseMultiReader<SegmentReader> {
|
|||
assert info.dir == dir;
|
||||
final SegmentReader reader = writer.readerPool.getReadOnlyClone(info, IOContext.READ);
|
||||
if (reader.numDocs() > 0 || writer.getKeepFullyDeletedSegments()) {
|
||||
reader.readerFinishedListeners = writer.getReaderFinishedListeners();
|
||||
readers.add(reader);
|
||||
infosUpto++;
|
||||
} else {
|
||||
|
@ -121,14 +111,12 @@ final class DirectoryReader extends BaseMultiReader<SegmentReader> {
|
|||
}
|
||||
}
|
||||
return new DirectoryReader(readers.toArray(new SegmentReader[readers.size()]),
|
||||
dir, writer, segmentInfos, writer.getConfig().getReaderTermsIndexDivisor(),
|
||||
applyAllDeletes, writer.getReaderFinishedListeners());
|
||||
dir, writer, segmentInfos, writer.getConfig().getReaderTermsIndexDivisor(), applyAllDeletes);
|
||||
}
|
||||
|
||||
/** This constructor is only used for {@link #doOpenIfChanged()} */
|
||||
static DirectoryReader open(Directory directory, IndexWriter writer, SegmentInfos infos, SegmentReader[] oldReaders,
|
||||
boolean doClone, int termInfosIndexDivisor, Collection<ReaderFinishedListener> readerFinishedListeners
|
||||
) throws IOException {
|
||||
boolean doClone, int termInfosIndexDivisor) throws IOException {
|
||||
// we put the old SegmentReaders in a map, that allows us
|
||||
// to lookup a reader using its segment name
|
||||
final Map<String,Integer> segmentReaders = new HashMap<String,Integer>();
|
||||
|
@ -168,7 +156,6 @@ final class DirectoryReader extends BaseMultiReader<SegmentReader> {
|
|||
|
||||
// this is a new reader; in case we hit an exception we can close it safely
|
||||
newReader = SegmentReader.get(infos.info(i), termInfosIndexDivisor, IOContext.READ);
|
||||
newReader.readerFinishedListeners = readerFinishedListeners;
|
||||
readerShared[i] = false;
|
||||
newReaders[i] = newReader;
|
||||
} else {
|
||||
|
@ -179,7 +166,6 @@ final class DirectoryReader extends BaseMultiReader<SegmentReader> {
|
|||
readerShared[i] = true;
|
||||
newReaders[i].incRef();
|
||||
} else {
|
||||
assert newReader.readerFinishedListeners == readerFinishedListeners;
|
||||
readerShared[i] = false;
|
||||
// Steal ref returned to us by reopenSegment:
|
||||
newReaders[i] = newReader;
|
||||
|
@ -212,9 +198,8 @@ final class DirectoryReader extends BaseMultiReader<SegmentReader> {
|
|||
if (prior != null) throw prior;
|
||||
}
|
||||
}
|
||||
return new DirectoryReader(newReaders,
|
||||
directory, writer, infos, termInfosIndexDivisor,
|
||||
false, readerFinishedListeners);
|
||||
return new DirectoryReader(newReaders, directory, writer,
|
||||
infos, termInfosIndexDivisor, false);
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
|
@ -242,7 +227,6 @@ final class DirectoryReader extends BaseMultiReader<SegmentReader> {
|
|||
public final synchronized Object clone() {
|
||||
try {
|
||||
DirectoryReader newReader = doOpenIfChanged((SegmentInfos) segmentInfos.clone(), true, writer);
|
||||
assert newReader.readerFinishedListeners != null;
|
||||
return newReader;
|
||||
} catch (Exception ex) {
|
||||
throw new RuntimeException(ex);
|
||||
|
@ -295,7 +279,6 @@ final class DirectoryReader extends BaseMultiReader<SegmentReader> {
|
|||
return null;
|
||||
}
|
||||
|
||||
reader.readerFinishedListeners = readerFinishedListeners;
|
||||
return reader;
|
||||
}
|
||||
|
||||
|
@ -325,7 +308,7 @@ final class DirectoryReader extends BaseMultiReader<SegmentReader> {
|
|||
}
|
||||
|
||||
private synchronized DirectoryReader doOpenIfChanged(SegmentInfos infos, boolean doClone, IndexWriter writer) throws CorruptIndexException, IOException {
|
||||
return DirectoryReader.open(directory, writer, infos, subReaders, doClone, termInfosIndexDivisor, readerFinishedListeners);
|
||||
return DirectoryReader.open(directory, writer, infos, subReaders, doClone, termInfosIndexDivisor);
|
||||
}
|
||||
|
||||
/** Version number when this IndexReader was opened. */
|
||||
|
|
|
@ -20,13 +20,11 @@ package org.apache.lucene.index;
|
|||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.MapBackedSet;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Map;
|
||||
import java.util.Comparator;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
/** A <code>FilterIndexReader</code> contains another IndexReader, which it
|
||||
* uses as its basic source of data, possibly transforming the data along the
|
||||
|
@ -282,7 +280,6 @@ public class FilterIndexReader extends IndexReader {
|
|||
public FilterIndexReader(IndexReader in) {
|
||||
super();
|
||||
this.in = in;
|
||||
readerFinishedListeners = new MapBackedSet<ReaderFinishedListener>(new ConcurrentHashMap<ReaderFinishedListener,Boolean>());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -408,18 +405,6 @@ public class FilterIndexReader extends IndexReader {
|
|||
return buffer.toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addReaderFinishedListener(ReaderFinishedListener listener) {
|
||||
super.addReaderFinishedListener(listener);
|
||||
in.addReaderFinishedListener(listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeReaderFinishedListener(ReaderFinishedListener listener) {
|
||||
super.removeReaderFinishedListener(listener);
|
||||
in.removeReaderFinishedListener(listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValues docValues(String field) throws IOException {
|
||||
ensureOpen();
|
||||
|
|
|
@ -24,16 +24,18 @@ import java.io.IOException;
|
|||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.DocumentStoredFieldVisitor;
|
||||
import org.apache.lucene.search.FieldCache; // javadocs
|
||||
import org.apache.lucene.search.SearcherManager; // javadocs
|
||||
import org.apache.lucene.store.*;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.MapBackedSet;
|
||||
import org.apache.lucene.util.CommandLineUtil;
|
||||
import org.apache.lucene.util.ReaderUtil; // for javadocs
|
||||
|
||||
|
@ -73,62 +75,40 @@ public abstract class IndexReader implements Cloneable,Closeable {
|
|||
|
||||
/**
|
||||
* A custom listener that's invoked when the IndexReader
|
||||
* is finished.
|
||||
*
|
||||
* <p>For a SegmentReader, this listener is called only
|
||||
* once all SegmentReaders sharing the same core are
|
||||
* closed. At this point it is safe for apps to evict
|
||||
* this reader from any caches keyed on {@link
|
||||
* #getCoreCacheKey}. This is the same interface that
|
||||
* {@link FieldCache} uses, internally, to evict
|
||||
* entries.</p>
|
||||
*
|
||||
* <p>For other readers, this listener is called when they
|
||||
* are closed.</p>
|
||||
* is closed.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public static interface ReaderFinishedListener {
|
||||
public void finished(IndexReader reader);
|
||||
public static interface ReaderClosedListener {
|
||||
public void onClose(IndexReader reader);
|
||||
}
|
||||
|
||||
// Impls must set this if they may call add/removeReaderFinishedListener:
|
||||
protected volatile Collection<ReaderFinishedListener> readerFinishedListeners;
|
||||
private final Set<ReaderClosedListener> readerClosedListeners =
|
||||
new MapBackedSet<ReaderClosedListener>(new ConcurrentHashMap<ReaderClosedListener, Boolean>());
|
||||
|
||||
/** Expert: adds a {@link ReaderFinishedListener}. The
|
||||
* provided listener is also added to any sub-readers, if
|
||||
* this is a composite reader. Also, any reader reopened
|
||||
* or cloned from this one will also copy the listeners at
|
||||
* the time of reopen.
|
||||
/** Expert: adds a {@link ReaderClosedListener}. The
|
||||
* provided listener will be invoked when this reader is closed.
|
||||
*
|
||||
* @lucene.experimental */
|
||||
public void addReaderFinishedListener(ReaderFinishedListener listener) {
|
||||
public final void addReaderClosedListener(ReaderClosedListener listener) {
|
||||
ensureOpen();
|
||||
readerFinishedListeners.add(listener);
|
||||
readerClosedListeners.add(listener);
|
||||
}
|
||||
|
||||
/** Expert: remove a previously added {@link ReaderFinishedListener}.
|
||||
/** Expert: remove a previously added {@link ReaderClosedListener}.
|
||||
*
|
||||
* @lucene.experimental */
|
||||
public void removeReaderFinishedListener(ReaderFinishedListener listener) {
|
||||
public final void removeReaderClosedListener(ReaderClosedListener listener) {
|
||||
ensureOpen();
|
||||
readerFinishedListeners.remove(listener);
|
||||
readerClosedListeners.remove(listener);
|
||||
}
|
||||
|
||||
protected void notifyReaderFinishedListeners() {
|
||||
// Defensive (should never be null -- all impls must set
|
||||
// this):
|
||||
if (readerFinishedListeners != null) {
|
||||
for(ReaderFinishedListener listener : readerFinishedListeners) {
|
||||
listener.finished(this);
|
||||
}
|
||||
private final void notifyReaderClosedListeners() {
|
||||
for(ReaderClosedListener listener : readerClosedListeners) {
|
||||
listener.onClose(this);
|
||||
}
|
||||
}
|
||||
|
||||
protected void readerFinished() {
|
||||
notifyReaderFinishedListeners();
|
||||
}
|
||||
|
||||
/**
|
||||
* Constants describing field properties, for example used for
|
||||
* {@link IndexReader#getFieldNames(FieldOption)}.
|
||||
|
@ -267,7 +247,7 @@ public abstract class IndexReader implements Cloneable,Closeable {
|
|||
refCount.incrementAndGet();
|
||||
}
|
||||
}
|
||||
readerFinished();
|
||||
notifyReaderClosedListeners();
|
||||
} else if (rc < 0) {
|
||||
throw new IllegalStateException("too many decRef calls: refCount is " + rc + " after decrement");
|
||||
}
|
||||
|
|
|
@ -29,7 +29,6 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
|
@ -53,7 +52,6 @@ import org.apache.lucene.util.Constants;
|
|||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
import org.apache.lucene.util.ThreadInterruptedException;
|
||||
import org.apache.lucene.util.MapBackedSet;
|
||||
import org.apache.lucene.util.TwoPhaseCommit;
|
||||
|
||||
/**
|
||||
|
@ -393,13 +391,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
return r;
|
||||
}
|
||||
|
||||
// Used for all SegmentReaders we open
|
||||
private final Collection<IndexReader.ReaderFinishedListener> readerFinishedListeners = new MapBackedSet<IndexReader.ReaderFinishedListener>(new ConcurrentHashMap<IndexReader.ReaderFinishedListener,Boolean>());
|
||||
|
||||
Collection<IndexReader.ReaderFinishedListener> getReaderFinishedListeners() throws IOException {
|
||||
return readerFinishedListeners;
|
||||
}
|
||||
|
||||
/** Holds shared SegmentReader instances. IndexWriter uses
|
||||
* SegmentReaders for 1) applying deletes, 2) doing
|
||||
* merges, 3) handing out a real-time reader. This pool
|
||||
|
@ -703,7 +694,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// synchronized
|
||||
// Returns a ref, which we xfer to readerMap:
|
||||
sr = SegmentReader.getRW(info, doOpenStores, context.context == IOContext.Context.MERGE ? -1 : config.getReaderTermsIndexDivisor(), context);
|
||||
sr.readerFinishedListeners = readerFinishedListeners;
|
||||
|
||||
if (info.dir == directory) {
|
||||
// Only pool if reader is not external
|
||||
|
|
|
@ -18,10 +18,6 @@ package org.apache.lucene.index;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
import org.apache.lucene.util.MapBackedSet;
|
||||
|
||||
/** An IndexReader which reads multiple indexes, appending
|
||||
* their content. */
|
||||
|
@ -45,7 +41,6 @@ public class MultiReader extends BaseMultiReader<IndexReader> {
|
|||
*/
|
||||
public MultiReader(IndexReader[] subReaders, boolean closeSubReaders) throws IOException {
|
||||
super(subReaders.clone());
|
||||
readerFinishedListeners = new MapBackedSet<ReaderFinishedListener>(new ConcurrentHashMap<ReaderFinishedListener,Boolean>());
|
||||
decrefOnClose = new boolean[subReaders.length];
|
||||
for (int i = 0; i < subReaders.length; i++) {
|
||||
if (!closeSubReaders) {
|
||||
|
@ -58,12 +53,10 @@ public class MultiReader extends BaseMultiReader<IndexReader> {
|
|||
}
|
||||
|
||||
// used only by openIfChaged
|
||||
private MultiReader(IndexReader[] subReaders, boolean[] decrefOnClose,
|
||||
Collection<ReaderFinishedListener> readerFinishedListeners)
|
||||
private MultiReader(IndexReader[] subReaders, boolean[] decrefOnClose)
|
||||
throws IOException {
|
||||
super(subReaders);
|
||||
this.decrefOnClose = decrefOnClose;
|
||||
this.readerFinishedListeners = readerFinishedListeners;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -125,7 +118,7 @@ public class MultiReader extends BaseMultiReader<IndexReader> {
|
|||
newDecrefOnClose[i] = true;
|
||||
}
|
||||
}
|
||||
return new MultiReader(newSubReaders, newDecrefOnClose, readerFinishedListeners);
|
||||
return new MultiReader(newSubReaders, newDecrefOnClose);
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
|
@ -169,20 +162,4 @@ public class MultiReader extends BaseMultiReader<IndexReader> {
|
|||
public long getVersion() {
|
||||
throw new UnsupportedOperationException("MultiReader does not support this method.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addReaderFinishedListener(ReaderFinishedListener listener) {
|
||||
super.addReaderFinishedListener(listener);
|
||||
for(IndexReader sub : subReaders) {
|
||||
sub.addReaderFinishedListener(listener);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeReaderFinishedListener(ReaderFinishedListener listener) {
|
||||
super.removeReaderFinishedListener(listener);
|
||||
for(IndexReader sub : subReaders) {
|
||||
sub.removeReaderFinishedListener(listener);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,11 +19,9 @@ package org.apache.lucene.index;
|
|||
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.MapBackedSet;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.*;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
|
||||
/** An IndexReader which reads multiple, parallel indexes. Each index added
|
||||
|
@ -70,7 +68,6 @@ public class ParallelReader extends IndexReader {
|
|||
public ParallelReader(boolean closeSubReaders) throws IOException {
|
||||
super();
|
||||
this.incRefReaders = !closeSubReaders;
|
||||
readerFinishedListeners = new MapBackedSet<ReaderFinishedListener>(new ConcurrentHashMap<ReaderFinishedListener,Boolean>());
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
|
@ -444,22 +441,6 @@ public class ParallelReader extends IndexReader {
|
|||
return topLevelReaderContext;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addReaderFinishedListener(ReaderFinishedListener listener) {
|
||||
super.addReaderFinishedListener(listener);
|
||||
for (IndexReader reader : readers) {
|
||||
reader.addReaderFinishedListener(listener);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeReaderFinishedListener(ReaderFinishedListener listener) {
|
||||
super.removeReaderFinishedListener(listener);
|
||||
for (IndexReader reader : readers) {
|
||||
reader.removeReaderFinishedListener(listener);
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: I suspect this is completely untested!!!!!
|
||||
@Override
|
||||
public DocValues docValues(String field) throws IOException {
|
||||
|
|
|
@ -18,8 +18,11 @@ package org.apache.lucene.index;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.lucene.index.SegmentReader.CoreClosedListener;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.NormsReader;
|
||||
import org.apache.lucene.index.codecs.PerDocProducer;
|
||||
|
@ -31,6 +34,7 @@ import org.apache.lucene.store.CompoundFileDirectory;
|
|||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.MapBackedSet;
|
||||
|
||||
/** Holds core readers that are shared (unchanged) when
|
||||
* SegmentReader is cloned or reopened */
|
||||
|
@ -63,7 +67,8 @@ final class SegmentCoreReaders {
|
|||
CompoundFileDirectory cfsReader;
|
||||
CompoundFileDirectory storeCFSReader;
|
||||
|
||||
|
||||
final Set<CoreClosedListener> coreClosedListeners =
|
||||
new MapBackedSet<CoreClosedListener>(new ConcurrentHashMap<CoreClosedListener, Boolean>());
|
||||
|
||||
SegmentCoreReaders(SegmentReader owner, Directory dir, SegmentInfo si, IOContext context, int termsIndexDivisor) throws IOException {
|
||||
|
||||
|
@ -133,9 +138,8 @@ final class SegmentCoreReaders {
|
|||
if (ref.decrementAndGet() == 0) {
|
||||
IOUtils.close(fields, perDocProducer, termVectorsReaderOrig,
|
||||
fieldsReaderOrig, cfsReader, storeCFSReader, norms);
|
||||
// Now, notify any ReaderFinished listeners:
|
||||
if (owner != null) {
|
||||
owner.notifyReaderFinishedListeners();
|
||||
for (CoreClosedListener listener : coreClosedListeners) {
|
||||
listener.onClose(owner);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.lucene.index.FieldInfo.IndexOptions;
|
|||
import org.apache.lucene.index.codecs.PerDocProducer;
|
||||
import org.apache.lucene.index.codecs.StoredFieldsReader;
|
||||
import org.apache.lucene.index.codecs.TermVectorsReader;
|
||||
import org.apache.lucene.search.FieldCache; // javadocs
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.BitVector;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
@ -413,16 +414,6 @@ public final class SegmentReader extends IndexReader implements Cloneable {
|
|||
public int getTermInfosIndexDivisor() {
|
||||
return core.termsIndexDivisor;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void readerFinished() {
|
||||
// Do nothing here -- we have more careful control on
|
||||
// when to notify that a SegmentReader has finished,
|
||||
// because a given core is shared across many cloned
|
||||
// SegmentReaders. We only notify once that core is no
|
||||
// longer used (all SegmentReaders sharing it have been
|
||||
// closed).
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValues docValues(String field) throws IOException {
|
||||
|
@ -474,7 +465,6 @@ public final class SegmentReader extends IndexReader implements Cloneable {
|
|||
core.incRef();
|
||||
clone.core = core;
|
||||
clone.pendingDeleteCount = pendingDeleteCount;
|
||||
clone.readerFinishedListeners = readerFinishedListeners;
|
||||
|
||||
if (!openReadOnly && hasChanges) {
|
||||
// My pending changes transfer to the new reader
|
||||
|
@ -607,4 +597,33 @@ public final class SegmentReader extends IndexReader implements Cloneable {
|
|||
pendingDeleteCount++;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Called when the shared core for this SegmentReader
|
||||
* is closed.
|
||||
* <p>
|
||||
* This listener is called only once all SegmentReaders
|
||||
* sharing the same core are closed. At this point it
|
||||
* is safe for apps to evict this reader from any caches
|
||||
* keyed on {@link #getCoreCacheKey}. This is the same
|
||||
* interface that {@link FieldCache} uses, internally,
|
||||
* to evict entries.</p>
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public static interface CoreClosedListener {
|
||||
public void onClose(SegmentReader owner);
|
||||
}
|
||||
|
||||
/** Expert: adds a CoreClosedListener to this reader's shared core */
|
||||
public void addCoreClosedListener(CoreClosedListener listener) {
|
||||
ensureOpen();
|
||||
core.coreClosedListeners.add(listener);
|
||||
}
|
||||
|
||||
/** Expert: removes a CoreClosedListener from this reader's shared core */
|
||||
public void removeCoreClosedListener(CoreClosedListener listener) {
|
||||
ensureOpen();
|
||||
core.coreClosedListeners.remove(listener);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -32,6 +32,7 @@ import org.apache.lucene.index.DocsEnum;
|
|||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.MultiFields;
|
||||
import org.apache.lucene.index.OrdTermState;
|
||||
import org.apache.lucene.index.SegmentReader;
|
||||
import org.apache.lucene.index.TermState;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
|
@ -145,11 +146,12 @@ class FieldCacheImpl implements FieldCache {
|
|||
*/
|
||||
static final class StopFillCacheException extends RuntimeException {
|
||||
}
|
||||
|
||||
final static IndexReader.ReaderFinishedListener purgeReader = new IndexReader.ReaderFinishedListener() {
|
||||
|
||||
// per-segment fieldcaches don't purge until the shared core closes.
|
||||
final static SegmentReader.CoreClosedListener purgeCore = new SegmentReader.CoreClosedListener() {
|
||||
// @Override -- not until Java 1.6
|
||||
public void finished(IndexReader reader) {
|
||||
FieldCache.DEFAULT.purge(reader);
|
||||
public void onClose(SegmentReader owner) {
|
||||
FieldCache.DEFAULT.purge(owner);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -188,7 +190,16 @@ class FieldCacheImpl implements FieldCache {
|
|||
// First time this reader is using FieldCache
|
||||
innerCache = new HashMap<Entry,Object>();
|
||||
readerCache.put(readerKey, innerCache);
|
||||
reader.addReaderFinishedListener(purgeReader);
|
||||
if (reader instanceof SegmentReader) {
|
||||
((SegmentReader) reader).addCoreClosedListener(purgeCore);
|
||||
} else {
|
||||
reader.addReaderClosedListener(new IndexReader.ReaderClosedListener() {
|
||||
@Override
|
||||
public void onClose(IndexReader reader) {
|
||||
FieldCache.DEFAULT.purge(reader);
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
if (innerCache.get(key) == null) {
|
||||
innerCache.put(key, value);
|
||||
|
@ -209,7 +220,16 @@ class FieldCacheImpl implements FieldCache {
|
|||
// First time this reader is using FieldCache
|
||||
innerCache = new HashMap<Entry,Object>();
|
||||
readerCache.put(readerKey, innerCache);
|
||||
reader.addReaderFinishedListener(purgeReader);
|
||||
if (reader instanceof SegmentReader) {
|
||||
((SegmentReader) reader).addCoreClosedListener(purgeCore);
|
||||
} else {
|
||||
reader.addReaderClosedListener(new IndexReader.ReaderClosedListener() {
|
||||
@Override
|
||||
public void onClose(IndexReader reader) {
|
||||
FieldCache.DEFAULT.purge(reader);
|
||||
}
|
||||
});
|
||||
}
|
||||
value = null;
|
||||
} else {
|
||||
value = innerCache.get(key);
|
||||
|
|
|
@ -36,7 +36,7 @@ import org.apache.lucene.analysis.Analyzer;
|
|||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
import org.apache.lucene.index.*;
|
||||
import org.apache.lucene.index.IndexReader.ReaderFinishedListener;
|
||||
import org.apache.lucene.index.IndexReader.ReaderClosedListener;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.PostingsFormat;
|
||||
import org.apache.lucene.index.codecs.appending.AppendingCodec;
|
||||
|
@ -1222,15 +1222,10 @@ public abstract class LuceneTestCase extends Assert {
|
|||
if (VERBOSE) {
|
||||
System.out.println("NOTE: newSearcher using ExecutorService with " + threads + " threads");
|
||||
}
|
||||
final IndexReader r0 = r;
|
||||
r.addReaderFinishedListener(new ReaderFinishedListener() {
|
||||
r.addReaderClosedListener(new ReaderClosedListener() {
|
||||
@Override
|
||||
public void finished(IndexReader reader) {
|
||||
// readerFinishedListener bogusly calls us with other random readers
|
||||
// so we must check that its *actually* the one we registered it on.
|
||||
if (reader == r0) {
|
||||
shutdownExecutorService(ex);
|
||||
}
|
||||
public void onClose(IndexReader reader) {
|
||||
shutdownExecutorService(ex);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
|
|
@ -25,8 +25,6 @@ import java.util.Collection;
|
|||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.HashMap;
|
||||
import java.util.Random;
|
||||
import org.junit.Assume;
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
|
@ -41,12 +39,9 @@ import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsFormat;
|
|||
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.FieldCache;
|
||||
import org.apache.lucene.search.similarities.DefaultSimilarity;
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.LockObtainFailedException;
|
||||
import org.apache.lucene.store.NoSuchDirectoryException;
|
||||
import org.apache.lucene.store.LockReleaseFailedException;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util._TestUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
@ -958,29 +953,26 @@ public class TestIndexReader extends LuceneTestCase {
|
|||
writer.commit();
|
||||
final IndexReader reader = writer.getReader();
|
||||
final int[] closeCount = new int[1];
|
||||
final IndexReader.ReaderFinishedListener listener = new IndexReader.ReaderFinishedListener() {
|
||||
public void finished(IndexReader reader) {
|
||||
final IndexReader.ReaderClosedListener listener = new IndexReader.ReaderClosedListener() {
|
||||
public void onClose(IndexReader reader) {
|
||||
closeCount[0]++;
|
||||
}
|
||||
};
|
||||
|
||||
reader.addReaderFinishedListener(listener);
|
||||
reader.addReaderClosedListener(listener);
|
||||
|
||||
reader.close();
|
||||
|
||||
// Just the top reader
|
||||
// Close the top reader, its the only one that should be closed
|
||||
assertEquals(1, closeCount[0]);
|
||||
writer.close();
|
||||
|
||||
// Now also the subs
|
||||
assertEquals(3, closeCount[0]);
|
||||
|
||||
IndexReader reader2 = IndexReader.open(dir);
|
||||
reader2.addReaderFinishedListener(listener);
|
||||
reader2.addReaderClosedListener(listener);
|
||||
|
||||
closeCount[0] = 0;
|
||||
reader2.close();
|
||||
assertEquals(3, closeCount[0]);
|
||||
assertEquals(1, closeCount[0]);
|
||||
dir.close();
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue