LUCENE-6299: IndexWriter was failing to enforce the 2.1 billion doc limit in one index

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1662571 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael McCandless 2015-02-26 21:23:53 +00:00
commit 62130ae70c
26 changed files with 625 additions and 147 deletions

View File

@ -717,6 +717,9 @@ Bug fixes
sorted (set) doc values instance at the same time.
(Tom Shally, Robert Muir, Adrien Grand)
* LUCENE-6299: IndexWriter was failing to enforce the 2.1 billion doc
limit. (Robert Muir, Mike McCandless)
API Changes
* LUCENE-6212: Deprecate IndexWriter APIs that accept per-document Analyzer.

View File

@ -64,24 +64,32 @@ public abstract class BaseCompositeReader<R extends IndexReader> extends Composi
* cloned and not protected for modification, the subclass is responsible
* to do this.
*/
protected BaseCompositeReader(R[] subReaders) {
protected BaseCompositeReader(R[] subReaders) throws IOException {
this.subReaders = subReaders;
this.subReadersList = Collections.unmodifiableList(Arrays.asList(subReaders));
starts = new int[subReaders.length + 1]; // build starts array
int maxDoc = 0, numDocs = 0;
long maxDoc = 0, numDocs = 0;
for (int i = 0; i < subReaders.length; i++) {
starts[i] = maxDoc;
starts[i] = (int) maxDoc;
final IndexReader r = subReaders[i];
maxDoc += r.maxDoc(); // compute maxDocs
if (maxDoc < 0 /* overflow */ || maxDoc > IndexWriter.getActualMaxDocs()) {
throw new IllegalArgumentException("Too many documents, composite IndexReaders cannot exceed " + IndexWriter.getActualMaxDocs());
}
numDocs += r.numDocs(); // compute numDocs
r.registerParentReader(this);
}
starts[subReaders.length] = maxDoc;
this.maxDoc = maxDoc;
this.numDocs = numDocs;
if (maxDoc > IndexWriter.getActualMaxDocs()) {
if (this instanceof DirectoryReader) {
// A single index has too many documents and it is corrupt (IndexWriter prevents this as of LUCENE-6299)
throw new CorruptIndexException("Too many documents: an index cannot exceed " + IndexWriter.getActualMaxDocs() + " but readers have total maxDoc=" + maxDoc, Arrays.toString(subReaders));
} else {
// Caller is building a MultiReader and it has too many documents; this case is just illegal arguments:
throw new IllegalArgumentException("Too many documents: composite IndexReaders cannot exceed " + IndexWriter.getActualMaxDocs() + " but readers have total maxDoc=" + maxDoc);
}
}
this.maxDoc = Math.toIntExact(maxDoc);
starts[subReaders.length] = this.maxDoc;
this.numDocs = Math.toIntExact(numDocs);
}
@Override

View File

@ -305,11 +305,11 @@ public abstract class DirectoryReader extends BaseCompositeReader<LeafReader> {
* Subclasses of {@code DirectoryReader} should take care to not allow
* modification of this internal array, e.g. {@link #doOpenIfChanged()}.
*/
protected DirectoryReader(Directory directory, LeafReader[] segmentReaders) {
protected DirectoryReader(Directory directory, LeafReader[] segmentReaders) throws IOException {
super(segmentReaders);
this.directory = directory;
}
/** Returns the directory this index resides in. */
public final Directory directory() {
// Don't ensureOpen here -- in certain cases, when a

View File

@ -226,12 +226,14 @@ final class DocumentsWriter implements Closeable, Accountable {
}
}
}
synchronized void lockAndAbortAll(IndexWriter indexWriter) {
/** Returns how many documents were aborted. */
synchronized long lockAndAbortAll(IndexWriter indexWriter) {
assert indexWriter.holdsFullFlushLock();
if (infoStream.isEnabled("DW")) {
infoStream.message("DW", "lockAndAbortAll");
}
long abortedDocCount = 0;
boolean success = false;
try {
deleteQueue.clear();
@ -239,12 +241,13 @@ final class DocumentsWriter implements Closeable, Accountable {
for (int i = 0; i < limit; i++) {
final ThreadState perThread = perThreadPool.getThreadState(i);
perThread.lock();
abortThreadState(perThread);
abortedDocCount += abortThreadState(perThread);
}
deleteQueue.clear();
flushControl.abortPendingFlushes();
flushControl.waitForFlush();
success = true;
return abortedDocCount;
} finally {
if (infoStream.isEnabled("DW")) {
infoStream.message("DW", "finished lockAndAbortAll success=" + success);
@ -255,22 +258,28 @@ final class DocumentsWriter implements Closeable, Accountable {
}
}
}
private final void abortThreadState(final ThreadState perThread) {
/** Returns how many documents were aborted. */
private final int abortThreadState(final ThreadState perThread) {
assert perThread.isHeldByCurrentThread();
if (perThread.isActive()) { // we might be closed
if (perThread.isInitialized()) {
try {
subtractFlushedNumDocs(perThread.dwpt.getNumDocsInRAM());
int abortedDocCount = perThread.dwpt.getNumDocsInRAM();
subtractFlushedNumDocs(abortedDocCount);
perThread.dwpt.abort();
return abortedDocCount;
} finally {
flushControl.doOnAbort(perThread);
}
} else {
flushControl.doOnAbort(perThread);
// This DWPT was never initialized so it has no indexed documents:
return 0;
}
} else {
assert closed;
return 0;
}
}

View File

@ -201,17 +201,18 @@ class DocumentsWriterPerThread {
/** Anything that will add N docs to the index should reserve first to
* make sure it's allowed. */
private void reserveDoc() {
private void reserveOneDoc() {
if (pendingNumDocs.incrementAndGet() > IndexWriter.getActualMaxDocs()) {
// Reserve failed
// Reserve failed: put the one doc back and throw exc:
pendingNumDocs.decrementAndGet();
throw new IllegalStateException("number of documents in the index cannot exceed " + IndexWriter.getActualMaxDocs());
throw new IllegalArgumentException("number of documents in the index cannot exceed " + IndexWriter.getActualMaxDocs());
}
}
public void updateDocument(IndexDocument doc, Analyzer analyzer, Term delTerm) throws IOException, AbortingException {
testPoint("DocumentsWriterPerThread addDocument start");
assert deleteQueue != null;
reserveOneDoc();
docState.doc = doc;
docState.analyzer = analyzer;
docState.docID = numDocsInRAM;
@ -224,7 +225,6 @@ class DocumentsWriterPerThread {
// document, so the counter will be "wrong" in that case, but
// it's very hard to fix (we can't easily distinguish aborting
// vs non-aborting exceptions):
reserveDoc();
boolean success = false;
try {
try {
@ -261,7 +261,7 @@ class DocumentsWriterPerThread {
// document, so the counter will be "wrong" in that case, but
// it's very hard to fix (we can't easily distinguish aborting
// vs non-aborting exceptions):
reserveDoc();
reserveOneDoc();
docState.doc = doc;
docState.docID = numDocsInRAM;
docCount++;

View File

@ -185,13 +185,13 @@ public class ExitableDirectoryReader extends FilterDirectoryReader {
* @param in DirectoryReader that this ExitableDirectoryReader wraps around to make it Exitable.
* @param queryTimeout The object to periodically check if the query should time out.
*/
public ExitableDirectoryReader(DirectoryReader in, QueryTimeout queryTimeout) {
public ExitableDirectoryReader(DirectoryReader in, QueryTimeout queryTimeout) throws IOException {
super(in, new ExitableSubReaderWrapper(queryTimeout));
this.queryTimeout = queryTimeout;
}
@Override
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) {
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
return new ExitableDirectoryReader(in, queryTimeout);
}
@ -200,7 +200,7 @@ public class ExitableDirectoryReader extends FilterDirectoryReader {
* can be used normally (e.g. passed to {@link DirectoryReader#openIfChanged(DirectoryReader)})
* and so on.
*/
public static DirectoryReader wrap(DirectoryReader in, QueryTimeout queryTimeout) {
public static DirectoryReader wrap(DirectoryReader in, QueryTimeout queryTimeout) throws IOException {
return new ExitableDirectoryReader(in, queryTimeout);
}

View File

@ -79,7 +79,7 @@ public abstract class FilterDirectoryReader extends DirectoryReader {
* @param in the DirectoryReader to filter
* @param wrapper the SubReaderWrapper to use to wrap subreaders
*/
public FilterDirectoryReader(DirectoryReader in, SubReaderWrapper wrapper) {
public FilterDirectoryReader(DirectoryReader in, SubReaderWrapper wrapper) throws IOException {
super(in.directory(), wrapper.wrap(in.getSequentialSubReaders()));
this.in = in;
}
@ -93,9 +93,9 @@ public abstract class FilterDirectoryReader extends DirectoryReader {
* @param in the DirectoryReader to wrap
* @return the wrapped DirectoryReader
*/
protected abstract DirectoryReader doWrapDirectoryReader(DirectoryReader in);
protected abstract DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException;
private final DirectoryReader wrapDirectoryReader(DirectoryReader in) {
private final DirectoryReader wrapDirectoryReader(DirectoryReader in) throws IOException {
return in == null ? null : doWrapDirectoryReader(in);
}

View File

@ -196,7 +196,7 @@ import org.apache.lucene.util.Version;
public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
/** Hard limit on maximum number of documents that may be added to the
* index. If you try to add more than this you'll hit {@code IllegalStateException}. */
* index. If you try to add more than this you'll hit {@code IllegalArgumentException}. */
// We defensively subtract 128 to be well below the lowest
// ArrayUtil.MAX_ARRAY_LENGTH on "typical" JVMs. We don't just use
// ArrayUtil.MAX_ARRAY_LENGTH here because this can vary across JVMs:
@ -839,6 +839,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
}
rollbackSegments = segmentInfos.createBackupSegmentInfos();
pendingNumDocs.set(segmentInfos.totalDocCount());
// start with previous field numbers, but new FieldInfos
globalFieldNumberMap = getFieldNumberMap();
@ -2055,13 +2056,16 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
*/
try {
synchronized (fullFlushLock) {
docWriter.lockAndAbortAll(this);
long abortedDocCount = docWriter.lockAndAbortAll(this);
pendingNumDocs.addAndGet(-abortedDocCount);
processEvents(false, true);
synchronized (this) {
try {
// Abort any running merges
abortMerges();
// Remove all segments
pendingNumDocs.addAndGet(-segmentInfos.totalDocCount());
segmentInfos.clear();
// Ask deleter to locate unreferenced files & remove them:
deleter.checkpoint(segmentInfos, false);
@ -2077,6 +2081,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
++changeCount;
segmentInfos.changed();
globalFieldNumberMap.clear();
success = true;
} finally {
docWriter.unlockAllAfterAbortAll(this);
@ -2317,6 +2322,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
* @throws IOException if there is a low-level IO error
* @throws LockObtainFailedException if we were unable to
* acquire the write lock in at least one directory
* @throws IllegalArgumentException if addIndexes would cause
* the index to exceed {@link #MAX_DOCS}
*/
public void addIndexes(Directory... dirs) throws IOException {
ensureOpen();
@ -2335,16 +2342,25 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
flush(false, true);
List<SegmentCommitInfo> infos = new ArrayList<>();
int totalDocCount = 0;
// long so we can detect int overflow:
long totalDocCount = 0;
List<SegmentInfos> commits = new ArrayList<>(dirs.length);
for (Directory dir : dirs) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "addIndexes: process directory " + dir);
}
SegmentInfos sis = SegmentInfos.readLatestCommit(dir); // read infos from dir
totalDocCount += sis.totalDocCount();
commits.add(sis);
}
// Best-effort up front check:
testReserveDocs(totalDocCount);
boolean success = false;
try {
for (Directory dir : dirs) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "addIndexes: process directory " + dir);
}
SegmentInfos sis = SegmentInfos.readLatestCommit(dir); // read infos from dir
totalDocCount += sis.totalDocCount();
for (SegmentInfos sis : commits) {
for (SegmentCommitInfo info : sis) {
assert !infos.contains(info): "dup info dir=" + info.info.dir + " name=" + info.info.name;
@ -2367,12 +2383,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
} finally {
if (!success) {
for(SegmentCommitInfo sipc : infos) {
for(String file : sipc.files()) {
try {
directory.deleteFile(file);
} catch (Throwable t) {
}
}
IOUtils.deleteFilesIgnoringExceptions(directory, sipc.files().toArray(new String[0]));
}
}
}
@ -2381,9 +2392,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
success = false;
try {
ensureOpen();
// Make sure adding the new documents to this index won't
// exceed the limit:
// Now reserve the docs, just before we update SIS:
reserveDocs(totalDocCount);
success = true;
} finally {
if (!success) {
@ -2441,10 +2453,14 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
* if the index is corrupt
* @throws IOException
* if there is a low-level IO error
* @throws IllegalArgumentException
* if addIndexes would cause the index to exceed {@link #MAX_DOCS}
*/
public void addIndexes(CodecReader... readers) throws IOException {
ensureOpen();
int numDocs = 0;
// long so we can detect int overflow:
long numDocs = 0;
try {
if (infoStream.isEnabled("IW")) {
@ -2456,12 +2472,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
for (CodecReader leaf : readers) {
numDocs += leaf.numDocs();
}
// Make sure adding the new documents to this index won't
// exceed the limit:
reserveDocs(numDocs);
final IOContext context = new IOContext(new MergeInfo(numDocs, -1, false, -1));
// Best-effort up front check:
testReserveDocs(numDocs);
final IOContext context = new IOContext(new MergeInfo(Math.toIntExact(numDocs), -1, false, -1));
// TODO: somehow we should fix this merge so it's
// abortable so that IW.close(false) is able to stop it
@ -2553,11 +2568,15 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
return;
}
ensureOpen();
// Now reserve the docs, just before we update SIS:
reserveDocs(numDocs);
segmentInfos.add(infoPerCommit);
checkpoint();
}
} catch (OutOfMemoryError oom) {
tragicEvent(oom, "addIndexes(IndexReader...)");
tragicEvent(oom, "addIndexes(CodecReader...)");
}
maybeMerge();
}
@ -4616,15 +4635,31 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
/** Anything that will add N docs to the index should reserve first to
* make sure it's allowed. This will throw {@code
* IllegalStateException} if it's not allowed. */
private void reserveDocs(int numDocs) {
if (pendingNumDocs.addAndGet(numDocs) > actualMaxDocs) {
// Reserve failed
pendingNumDocs.addAndGet(-numDocs);
throw new IllegalStateException("number of documents in the index cannot exceed " + actualMaxDocs);
* IllegalArgumentException} if it's not allowed. */
private void reserveDocs(long addedNumDocs) {
assert addedNumDocs >= 0;
if (pendingNumDocs.addAndGet(addedNumDocs) > actualMaxDocs) {
// Reserve failed: put the docs back and throw exc:
pendingNumDocs.addAndGet(-addedNumDocs);
tooManyDocs(addedNumDocs);
}
}
/** Does a best-effort check, that the current index would accept this many additional docs, but does not actually reserve them.
*
* @throws IllegalArgumentException if there would be too many docs */
private void testReserveDocs(long addedNumDocs) {
assert addedNumDocs >= 0;
if (pendingNumDocs.get() + addedNumDocs > actualMaxDocs) {
tooManyDocs(addedNumDocs);
}
}
private void tooManyDocs(long addedNumDocs) {
assert addedNumDocs >= 0;
throw new IllegalArgumentException("number of documents in the index cannot exceed " + actualMaxDocs + " (current document count is " + pendingNumDocs.get() + "; added numDocs is " + addedNumDocs + ")");
}
/** Wraps the incoming {@link Directory} so that we assign a per-thread
* {@link MergeRateLimiter} to all created {@link IndexOutput}s. */
private Directory addMergeRateLimiters(Directory in) {

View File

@ -45,7 +45,7 @@ public class MultiReader extends BaseCompositeReader<IndexReader> {
* <p>Note that all subreaders are closed if this Multireader is closed.</p>
* @param subReaders set of (sub)readers
*/
public MultiReader(IndexReader... subReaders) {
public MultiReader(IndexReader... subReaders) throws IOException {
this(subReaders, true);
}
@ -55,7 +55,7 @@ public class MultiReader extends BaseCompositeReader<IndexReader> {
* @param closeSubReaders indicates whether the subreaders should be closed
* when this MultiReader is closed
*/
public MultiReader(IndexReader[] subReaders, boolean closeSubReaders) {
public MultiReader(IndexReader[] subReaders, boolean closeSubReaders) throws IOException {
super(subReaders.clone());
this.closeSubReaders = closeSubReaders;
if (!closeSubReaders) {

View File

@ -284,6 +284,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
if (numSegments < 0) {
throw new CorruptIndexException("invalid segment count: " + numSegments, input);
}
long totalDocs = 0;
for (int seg = 0; seg < numSegments; seg++) {
String segName = input.readString();
final byte segmentID[];
@ -297,6 +298,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
Codec codec = Codec.forName(input.readString());
SegmentInfo info = codec.segmentInfoFormat().read(directory, segName, segmentID, IOContext.READ);
info.setCodec(codec);
totalDocs += info.getDocCount();
long delGen = input.readLong();
int delCount = input.readInt();
if (delCount < 0 || delCount > info.getDocCount()) {
@ -323,6 +325,11 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
CodecUtil.checkFooter(input);
// LUCENE-6299: check we are in bounds
if (totalDocs > IndexWriter.getActualMaxDocs()) {
throw new CorruptIndexException("Too many documents: an index cannot exceed " + IndexWriter.getActualMaxDocs() + " but readers have total maxDoc=" + totalDocs, input);
}
return infos;
}
}
@ -722,11 +729,13 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
/** Returns sum of all segment's docCounts. Note that
* this does not include deletions */
public int totalDocCount() {
int count = 0;
long count = 0;
for(SegmentCommitInfo info : this) {
count += info.info.getDocCount();
}
return count;
// we should never hit this, checks should happen elsewhere...
assert count <= IndexWriter.getActualMaxDocs();
return Math.toIntExact(count);
}
/** Call this before committing if changes have been made to the

View File

@ -38,7 +38,7 @@ final class StandardDirectoryReader extends DirectoryReader {
/** called only from static open() methods */
StandardDirectoryReader(Directory directory, LeafReader[] readers, IndexWriter writer,
SegmentInfos sis, boolean applyAllDeletes) {
SegmentInfos sis, boolean applyAllDeletes) throws IOException {
super(directory, readers);
this.writer = writer;
this.segmentInfos = sis;
@ -52,18 +52,23 @@ final class StandardDirectoryReader extends DirectoryReader {
protected DirectoryReader doBody(String segmentFileName) throws IOException {
SegmentInfos sis = SegmentInfos.readCommit(directory, segmentFileName);
final SegmentReader[] readers = new SegmentReader[sis.size()];
for (int i = sis.size()-1; i >= 0; i--) {
boolean success = false;
try {
boolean success = false;
try {
for (int i = sis.size()-1; i >= 0; i--) {
readers[i] = new SegmentReader(sis.info(i), IOContext.READ);
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(readers);
}
}
// This may throw CorruptIndexException if there are too many docs, so
// it must be inside try clause so we close readers in that case:
DirectoryReader reader = new StandardDirectoryReader(directory, readers, null, sis, false);
success = true;
return reader;
} finally {
if (success == false) {
IOUtils.closeWhileHandlingException(readers);
}
}
return new StandardDirectoryReader(directory, readers, null, sis, false);
}
}.run(commit);
}

View File

@ -191,7 +191,7 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
}
private class ParallelLeafDirectoryReader extends FilterDirectoryReader {
public ParallelLeafDirectoryReader(DirectoryReader in) {
public ParallelLeafDirectoryReader(DirectoryReader in) throws IOException {
super(in, new FilterDirectoryReader.SubReaderWrapper() {
final long currentSchemaGen = getCurrentSchemaGen();
@Override
@ -207,7 +207,7 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
}
@Override
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) {
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
return new ParallelLeafDirectoryReader(in);
}

View File

@ -17,8 +17,10 @@ package org.apache.lucene.index;
* limitations under the License.
*/
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.concurrent.CountDownLatch;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
@ -28,11 +30,13 @@ import org.apache.lucene.search.SortField;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FilterDirectory;
import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.store.NoLockFactory;
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.TimeUnits;
import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
@SuppressCodecs({ "SimpleText", "Memory", "Direct" })
@ -93,7 +97,7 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
try {
w.addDocument(new Document());
fail("didn't hit exception");
} catch (IllegalStateException ise) {
} catch (IllegalArgumentException iae) {
// expected
}
w.close();
@ -116,7 +120,7 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
try {
w.addDocuments(Collections.singletonList(new Document()));
fail("didn't hit exception");
} catch (IllegalStateException ise) {
} catch (IllegalArgumentException iae) {
// expected
}
w.close();
@ -139,7 +143,7 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
try {
w.updateDocument(new Term("field", "foo"), new Document());
fail("didn't hit exception");
} catch (IllegalStateException ise) {
} catch (IllegalArgumentException iae) {
// expected
}
w.close();
@ -162,7 +166,7 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
try {
w.updateDocuments(new Term("field", "foo"), Collections.singletonList(new Document()));
fail("didn't hit exception");
} catch (IllegalStateException ise) {
} catch (IllegalArgumentException iae) {
// expected
}
w.close();
@ -201,7 +205,7 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
try {
w.addDocument(new Document());
fail("didn't hit exception");
} catch (IllegalStateException ise) {
} catch (IllegalArgumentException iae) {
// expected
}
w.close();
@ -247,7 +251,7 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
try {
w.addDocument(new Document());
fail("didn't hit exception");
} catch (IllegalStateException ise) {
} catch (IllegalArgumentException iae) {
// expected
}
w.close();
@ -273,7 +277,7 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
try {
w2.addIndexes(new Directory[] {dir});
fail("didn't hit exception");
} catch (IllegalStateException ise) {
} catch (IllegalArgumentException iae) {
// expected
}
assertEquals(1, w2.maxDoc());
@ -281,7 +285,7 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
try {
TestUtil.addIndexesSlowly(w2, ir);
fail("didn't hit exception");
} catch (IllegalStateException ise) {
} catch (IllegalArgumentException iae) {
// expected
}
w2.close();
@ -369,6 +373,103 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
dir.close();
dir2.close();
}
/**
* LUCENE-6299: Test if addindexes(Dir[]) prevents exceeding max docs.
*/
public void testAddTooManyIndexesDir() throws Exception {
// we cheat and add the same one over again... IW wants a write lock on each
Directory dir = newDirectory(random(), NoLockFactory.INSTANCE);
Document doc = new Document();
IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
for (int i = 0; i < 100000; i++) {
w.addDocument(doc);
}
w.forceMerge(1);
w.commit();
w.close();
// wrap this with disk full, so test fails faster and doesn't fill up real disks.
MockDirectoryWrapper dir2 = newMockDirectory();
w = new IndexWriter(dir2, new IndexWriterConfig(null));
w.commit(); // don't confuse checkindex
dir2.setMaxSizeInBytes(dir2.sizeInBytes() + 65536); // 64KB
Directory dirs[] = new Directory[1 + (IndexWriter.MAX_DOCS / 100000)];
for (int i = 0; i < dirs.length; i++) {
// bypass iw check for duplicate dirs
dirs[i] = new FilterDirectory(dir) {};
}
try {
w.addIndexes(dirs);
fail("didn't get expected exception");
} catch (IllegalArgumentException expected) {
// pass
} catch (IOException fakeDiskFull) {
final Exception e;
if (fakeDiskFull.getMessage() != null && fakeDiskFull.getMessage().startsWith("fake disk full")) {
e = new RuntimeException("test failed: IW checks aren't working and we are executing addIndexes");
e.addSuppressed(fakeDiskFull);
} else {
e = fakeDiskFull;
}
throw e;
}
w.close();
dir.close();
dir2.close();
}
/**
* LUCENE-6299: Test if addindexes(CodecReader[]) prevents exceeding max docs.
*/
public void testAddTooManyIndexesCodecReader() throws Exception {
// we cheat and add the same one over again... IW wants a write lock on each
Directory dir = newDirectory(random(), NoLockFactory.INSTANCE);
Document doc = new Document();
IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
for (int i = 0; i < 100000; i++) {
w.addDocument(doc);
}
w.forceMerge(1);
w.commit();
w.close();
// wrap this with disk full, so test fails faster and doesn't fill up real disks.
MockDirectoryWrapper dir2 = newMockDirectory();
w = new IndexWriter(dir2, new IndexWriterConfig(null));
w.commit(); // don't confuse checkindex
dir2.setMaxSizeInBytes(dir2.sizeInBytes() + 65536); // 64KB
IndexReader r = DirectoryReader.open(dir);
CodecReader segReader = (CodecReader) r.leaves().get(0).reader();
CodecReader readers[] = new CodecReader[1 + (IndexWriter.MAX_DOCS / 100000)];
for (int i = 0; i < readers.length; i++) {
readers[i] = segReader;
}
try {
w.addIndexes(readers);
fail("didn't get expected exception");
} catch (IllegalArgumentException expected) {
// pass
} catch (IOException fakeDiskFull) {
final Exception e;
if (fakeDiskFull.getMessage() != null && fakeDiskFull.getMessage().startsWith("fake disk full")) {
e = new RuntimeException("test failed: IW checks aren't working and we are executing addIndexes");
e.addSuppressed(fakeDiskFull);
} else {
e = fakeDiskFull;
}
throw e;
}
r.close();
w.close();
dir.close();
dir2.close();
}
public void testTooLargeMaxDocs() throws Exception {
try {
@ -378,4 +479,244 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
// expected
}
}
// LUCENE-6299
public void testDeleteAll() throws Exception {
setIndexWriterMaxDocs(1);
try {
Directory dir = newDirectory();
IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
w.addDocument(new Document());
try {
w.addDocument(new Document());
fail("didn't hit exception");
} catch (IllegalArgumentException iae) {
// expected
}
w.deleteAll();
w.addDocument(new Document());
try {
w.addDocument(new Document());
fail("didn't hit exception");
} catch (IllegalArgumentException iae) {
// expected
}
w.close();
dir.close();
} finally {
restoreIndexWriterMaxDocs();
}
}
// LUCENE-6299
public void testDeleteAllAfterFlush() throws Exception {
setIndexWriterMaxDocs(2);
try {
Directory dir = newDirectory();
IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
w.addDocument(new Document());
w.getReader().close();
w.addDocument(new Document());
try {
w.addDocument(new Document());
fail("didn't hit exception");
} catch (IllegalArgumentException iae) {
// expected
}
w.deleteAll();
w.addDocument(new Document());
w.addDocument(new Document());
try {
w.addDocument(new Document());
fail("didn't hit exception");
} catch (IllegalArgumentException iae) {
// expected
}
w.close();
dir.close();
} finally {
restoreIndexWriterMaxDocs();
}
}
// LUCENE-6299
public void testDeleteAllAfterCommit() throws Exception {
setIndexWriterMaxDocs(2);
try {
Directory dir = newDirectory();
IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
w.addDocument(new Document());
w.commit();
w.addDocument(new Document());
try {
w.addDocument(new Document());
fail("didn't hit exception");
} catch (IllegalArgumentException iae) {
// expected
}
w.deleteAll();
w.addDocument(new Document());
w.addDocument(new Document());
try {
w.addDocument(new Document());
fail("didn't hit exception");
} catch (IllegalArgumentException iae) {
// expected
}
w.close();
dir.close();
} finally {
restoreIndexWriterMaxDocs();
}
}
// LUCENE-6299
public void testDeleteAllMultipleThreads() throws Exception {
int limit = TestUtil.nextInt(random(), 2, 10);
setIndexWriterMaxDocs(limit);
try {
Directory dir = newDirectory();
IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
CountDownLatch startingGun = new CountDownLatch(1);
Thread[] threads = new Thread[limit];
for(int i=0;i<limit;i++) {
threads[i] = new Thread() {
@Override
public void run() {
try {
startingGun.await();
w.addDocument(new Document());
} catch (Exception e) {
throw new RuntimeException(e);
}
}
};
threads[i].start();
}
startingGun.countDown();
for(Thread thread : threads) {
thread.join();
}
try {
w.addDocument(new Document());
fail("didn't hit exception");
} catch (IllegalArgumentException iae) {
// expected
}
w.deleteAll();
for(int i=0;i<limit;i++) {
w.addDocument(new Document());
}
try {
w.addDocument(new Document());
fail("didn't hit exception");
} catch (IllegalArgumentException iae) {
// expected
}
w.close();
dir.close();
} finally {
restoreIndexWriterMaxDocs();
}
}
// LUCENE-6299
public void testDeleteAllAfterClose() throws Exception {
setIndexWriterMaxDocs(2);
try {
Directory dir = newDirectory();
IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
w.addDocument(new Document());
w.close();
w = new IndexWriter(dir, new IndexWriterConfig(null));
w.addDocument(new Document());
try {
w.addDocument(new Document());
fail("didn't hit exception");
} catch (IllegalArgumentException iae) {
// expected
}
w.deleteAll();
w.addDocument(new Document());
w.addDocument(new Document());
try {
w.addDocument(new Document());
fail("didn't hit exception");
} catch (IllegalArgumentException iae) {
// expected
}
w.close();
dir.close();
} finally {
restoreIndexWriterMaxDocs();
}
}
// LUCENE-6299
public void testAcrossTwoIndexWriters() throws Exception {
setIndexWriterMaxDocs(1);
try {
Directory dir = newDirectory();
IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
w.addDocument(new Document());
w.close();
w = new IndexWriter(dir, new IndexWriterConfig(null));
try {
w.addDocument(new Document());
fail("didn't hit exception");
} catch (IllegalArgumentException iae) {
// expected
}
w.close();
dir.close();
} finally {
restoreIndexWriterMaxDocs();
}
}
// LUCENE-6299
public void testCorruptIndexExceptionTooLarge() throws Exception {
Directory dir = newDirectory();
IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
w.addDocument(new Document());
w.addDocument(new Document());
w.close();
setIndexWriterMaxDocs(1);
try {
DirectoryReader.open(dir);
fail("didn't hit exception");
} catch (CorruptIndexException cie) {
// expected
} finally {
restoreIndexWriterMaxDocs();
}
dir.close();
}
// LUCENE-6299
public void testCorruptIndexExceptionTooLargeWriter() throws Exception {
Directory dir = newDirectory();
IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
w.addDocument(new Document());
w.addDocument(new Document());
w.close();
setIndexWriterMaxDocs(1);
try {
new IndexWriter(dir, new IndexWriterConfig(null));
fail("didn't hit exception");
} catch (CorruptIndexException cie) {
// expected
} finally {
restoreIndexWriterMaxDocs();
}
dir.close();
}
}

View File

@ -457,7 +457,7 @@ public class TestSearcherManager extends ThreadedIndexingAndSearchingTestCase {
}
private static class MyFilterDirectoryReader extends FilterDirectoryReader {
public MyFilterDirectoryReader(DirectoryReader in) {
public MyFilterDirectoryReader(DirectoryReader in) throws IOException {
super(in,
new FilterDirectoryReader.SubReaderWrapper() {
@Override
@ -470,7 +470,7 @@ public class TestSearcherManager extends ThreadedIndexingAndSearchingTestCase {
}
@Override
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) {
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
return new MyFilterDirectoryReader(in);
}
}

View File

@ -19,8 +19,10 @@ package org.apache.lucene.store;
import java.io.IOException;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LuceneTestCase;
@ -91,4 +93,16 @@ public class TestMockDirectoryWrapper extends LuceneTestCase {
dir.close();
}
public void testMDWinsideOfMDW() throws Exception {
// add MDW inside another MDW
Directory dir = new MockDirectoryWrapper(random(), newMockDirectory());
RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
for (int i = 0; i < 20; i++) {
iw.addDocument(new Document());
}
iw.commit();
iw.close();
dir.close();
}
}

View File

@ -31,7 +31,6 @@ import java.util.SortedSet;
import java.util.TreeSet;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexReader;
@ -39,6 +38,7 @@ import org.apache.lucene.index.IndexReaderContext;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.MultiReader;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.ReaderUtil;
import org.apache.lucene.index.StoredFieldVisitor;
import org.apache.lucene.index.Term;
@ -100,7 +100,14 @@ public class PostingsHighlighter {
// unnecessary.
/** for rewriting: we don't want slow processing from MTQs */
private static final IndexReader EMPTY_INDEXREADER = new MultiReader();
private static final IndexReader EMPTY_INDEXREADER;
static {
try {
EMPTY_INDEXREADER = new MultiReader();
} catch (IOException bogus) {
throw new RuntimeException(bogus);
}
}
/** Default maximum content size to process. Typically snippets
* closer to the beginning of the document better summarize its content */

View File

@ -146,14 +146,14 @@ public class UninvertingReader extends FilterLeafReader {
* can be used normally (e.g. passed to {@link DirectoryReader#openIfChanged(DirectoryReader)})
* and so on.
*/
public static DirectoryReader wrap(DirectoryReader in, final Map<String,Type> mapping) {
public static DirectoryReader wrap(DirectoryReader in, final Map<String,Type> mapping) throws IOException {
return new UninvertingDirectoryReader(in, mapping);
}
static class UninvertingDirectoryReader extends FilterDirectoryReader {
final Map<String,Type> mapping;
public UninvertingDirectoryReader(DirectoryReader in, final Map<String,Type> mapping) {
public UninvertingDirectoryReader(DirectoryReader in, final Map<String,Type> mapping) throws IOException {
super(in, new FilterDirectoryReader.SubReaderWrapper() {
@Override
public LeafReader wrap(LeafReader reader) {
@ -164,7 +164,7 @@ public class UninvertingReader extends FilterLeafReader {
}
@Override
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) {
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
return new UninvertingDirectoryReader(in, mapping);
}
}

View File

@ -17,6 +17,8 @@ package org.apache.lucene.index;
* limitations under the License.
*/
import java.io.IOException;
/**
* A {@link DirectoryReader} that wraps all its subreaders with
* {@link AssertingLeafReader}
@ -30,12 +32,12 @@ public class AssertingDirectoryReader extends FilterDirectoryReader {
}
}
public AssertingDirectoryReader(DirectoryReader in) {
public AssertingDirectoryReader(DirectoryReader in) throws IOException {
super(in, new AssertingSubReaderWrapper());
}
@Override
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) {
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
return new AssertingDirectoryReader(in);
}

View File

@ -599,7 +599,7 @@ public abstract class BaseStoredFieldsFormatTestCase extends BaseIndexFileFormat
private static class DummyFilterDirectoryReader extends FilterDirectoryReader {
public DummyFilterDirectoryReader(DirectoryReader in) {
public DummyFilterDirectoryReader(DirectoryReader in) throws IOException {
super(in, new SubReaderWrapper() {
@Override
public LeafReader wrap(LeafReader reader) {
@ -609,7 +609,7 @@ public abstract class BaseStoredFieldsFormatTestCase extends BaseIndexFileFormat
}
@Override
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) {
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
return new DummyFilterDirectoryReader(in);
}

View File

@ -17,6 +17,7 @@ package org.apache.lucene.index;
* limitations under the License.
*/
import java.io.IOException;
import java.util.Random;
/**
@ -38,12 +39,12 @@ public class MismatchedDirectoryReader extends FilterDirectoryReader {
}
}
public MismatchedDirectoryReader(DirectoryReader in, Random random) {
public MismatchedDirectoryReader(DirectoryReader in, Random random) throws IOException {
super(in, new MismatchedSubReaderWrapper(random));
}
@Override
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) {
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
return new AssertingDirectoryReader(in);
}
}

View File

@ -22,6 +22,7 @@ import java.util.List;
import java.util.Random;
import junit.framework.Assert;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.AllDeletedFilterReader;
@ -133,7 +134,7 @@ public class QueryUtils {
public static class FCInvisibleMultiReader extends MultiReader {
private final Object cacheKey = new Object();
public FCInvisibleMultiReader(IndexReader... readers) {
public FCInvisibleMultiReader(IndexReader... readers) throws IOException {
super(readers);
}

View File

@ -28,13 +28,13 @@ import org.apache.lucene.util.TestUtil;
// do NOT make any methods in this class synchronized, volatile
// do NOT import anything from the concurrency package.
// no randoms, no nothing.
public class BaseDirectoryWrapper extends FilterDirectory {
public abstract class BaseDirectoryWrapper extends FilterDirectory {
private boolean checkIndexOnClose = true;
private boolean crossCheckTermVectorsOnClose = true;
protected volatile boolean isOpen = true;
public BaseDirectoryWrapper(Directory delegate) {
protected BaseDirectoryWrapper(Directory delegate) {
super(delegate);
}
@ -72,10 +72,4 @@ public class BaseDirectoryWrapper extends FilterDirectory {
public boolean getCrossCheckTermVectorsOnClose() {
return crossCheckTermVectorsOnClose;
}
// why does this class override this method?
@Override
public void copyFrom(Directory from, String src, String dest, IOContext context) throws IOException {
in.copyFrom(from, src, dest, context);
}
}

View File

@ -209,23 +209,6 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
useSlowOpenClosers = v;
}
/**
* Returns true if {@link #in} must sync its files.
* Currently, only {@link NRTCachingDirectory} requires sync'ing its files
* because otherwise they are cached in an internal {@link RAMDirectory}. If
* other directories require that too, they should be added to this method.
*/
private boolean mustSync() {
Directory delegate = in;
while (delegate instanceof FilterDirectory) {
if (delegate instanceof NRTCachingDirectory) {
return true;
}
delegate = ((FilterDirectory) delegate).getDelegate();
}
return delegate instanceof NRTCachingDirectory;
}
@Override
public synchronized void sync(Collection<String> names) throws IOException {
maybeYield();
@ -233,16 +216,13 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
if (crashed) {
throw new IOException("cannot sync after crash");
}
// don't wear out our hardware so much in tests.
if (LuceneTestCase.rarely(randomState) || mustSync()) {
for (String name : names) {
// randomly fail with IOE on any file
maybeThrowIOException(name);
in.sync(Collections.singleton(name));
unSyncedFiles.remove(name);
}
} else {
unSyncedFiles.removeAll(names);
// always pass thru fsync, directories rely on this.
// 90% of time, we use DisableFsyncFS which omits the real calls.
for (String name : names) {
// randomly fail with IOE on any file
maybeThrowIOException(name);
in.sync(Collections.singleton(name));
unSyncedFiles.remove(name);
}
}
@ -1051,20 +1031,38 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
public boolean isLocked() throws IOException {
return delegateLock.isLocked();
}
}
// TODO: why does this class override this method?
// we should use the default implementation so all of our checks work?
@Override
public synchronized void copyFrom(Directory from, String src, String dest, IOContext context) throws IOException {
maybeYield();
// randomize the IOContext here?
in.copyFrom(from, src, dest, context);
}
}
/** Use this when throwing fake {@code IOException},
* e.g. from {@link MockDirectoryWrapper.Failure}. */
public static class FakeIOException extends IOException {
}
@Override
public String toString() {
if (maxSize != 0) {
return "MockDirectoryWrapper(" + in + ", current=" + maxUsedSize + ",max=" + maxSize + ")";
} else {
return super.toString();
}
}
// don't override optional methods like copyFrom: we need the default impl for things like disk
// full checks. we randomly exercise "raw" directories anyway. We ensure default impls are used:
@Override
public final ChecksumIndexInput openChecksumInput(String name, IOContext context) throws IOException {
return super.openChecksumInput(name, context);
}
@Override
public final void copyFrom(Directory from, String src, String dest, IOContext context) throws IOException {
super.copyFrom(from, src, dest, context);
}
@Override
protected final void ensureOpen() throws AlreadyClosedException {
super.ensureOpen();
}
}

View File

@ -0,0 +1,49 @@
package org.apache.lucene.store;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
/**
* Delegates all operations, even optional ones, to the wrapped directory.
* <p>
* This class is used if you want the most realistic testing, but still
* with a checkindex on close. If you want asserts and evil things,
* use MockDirectoryWrapper instead.
*/
public final class RawDirectoryWrapper extends BaseDirectoryWrapper {
public RawDirectoryWrapper(Directory delegate) {
super(delegate);
}
@Override
public void copyFrom(Directory from, String src, String dest, IOContext context) throws IOException {
in.copyFrom(from, src, dest, context);
}
@Override
public ChecksumIndexInput openChecksumInput(String name, IOContext context) throws IOException {
return in.openChecksumInput(name, context);
}
@Override
protected void ensureOpen() throws AlreadyClosedException {
in.ensureOpen();
}
}

View File

@ -82,6 +82,7 @@ import com.carrotsearch.randomizedtesting.rules.NoClassHooksShadowingRule;
import com.carrotsearch.randomizedtesting.rules.NoInstanceHooksOverridesRule;
import com.carrotsearch.randomizedtesting.rules.StaticFieldsInvariantRule;
import com.carrotsearch.randomizedtesting.rules.SystemPropertiesInvariantRule;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Field;
@ -110,6 +111,7 @@ import org.apache.lucene.store.MergeInfo;
import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.store.MockDirectoryWrapper.Throttling;
import org.apache.lucene.store.NRTCachingDirectory;
import org.apache.lucene.store.RawDirectoryWrapper;
import org.apache.lucene.util.automaton.AutomatonTestUtil;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.automaton.RegExp;
@ -1317,7 +1319,7 @@ public abstract class LuceneTestCase extends Assert {
}
if (bare) {
BaseDirectoryWrapper base = new BaseDirectoryWrapper(directory);
BaseDirectoryWrapper base = new RawDirectoryWrapper(directory);
closeAfterSuite(new CloseableDirectory(base, suiteFailureMarker));
return base;
} else {

View File

@ -67,13 +67,13 @@ import org.apache.lucene.uninverting.UninvertingReader;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.core.DirectoryFactory;
import org.apache.solr.core.DirectoryFactory.DirContext;
import org.apache.solr.core.DirectoryFactory;
import org.apache.solr.core.SolrConfig;
import org.apache.solr.core.SolrCore;
import org.apache.solr.core.SolrInfoMBean;
@ -167,7 +167,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn
// TODO: wrap elsewhere and return a "map" from the schema that overrides get() ?
// this reader supports reopen
private static DirectoryReader wrapReader(SolrCore core, DirectoryReader reader) {
private static DirectoryReader wrapReader(SolrCore core, DirectoryReader reader) throws IOException {
assert reader != null;
return ExitableDirectoryReader.wrap
(UninvertingReader.wrap(reader, core.getLatestSchema().getUninversionMap(reader)),