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. sorted (set) doc values instance at the same time.
(Tom Shally, Robert Muir, Adrien Grand) (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 API Changes
* LUCENE-6212: Deprecate IndexWriter APIs that accept per-document Analyzer. * 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 * cloned and not protected for modification, the subclass is responsible
* to do this. * to do this.
*/ */
protected BaseCompositeReader(R[] subReaders) { protected BaseCompositeReader(R[] subReaders) throws IOException {
this.subReaders = subReaders; this.subReaders = subReaders;
this.subReadersList = Collections.unmodifiableList(Arrays.asList(subReaders)); this.subReadersList = Collections.unmodifiableList(Arrays.asList(subReaders));
starts = new int[subReaders.length + 1]; // build starts array 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++) { for (int i = 0; i < subReaders.length; i++) {
starts[i] = maxDoc; starts[i] = (int) maxDoc;
final IndexReader r = subReaders[i]; final IndexReader r = subReaders[i];
maxDoc += r.maxDoc(); // compute maxDocs 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 numDocs += r.numDocs(); // compute numDocs
r.registerParentReader(this); r.registerParentReader(this);
} }
starts[subReaders.length] = maxDoc;
this.maxDoc = maxDoc; if (maxDoc > IndexWriter.getActualMaxDocs()) {
this.numDocs = numDocs; 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 @Override

View File

@ -305,7 +305,7 @@ public abstract class DirectoryReader extends BaseCompositeReader<LeafReader> {
* Subclasses of {@code DirectoryReader} should take care to not allow * Subclasses of {@code DirectoryReader} should take care to not allow
* modification of this internal array, e.g. {@link #doOpenIfChanged()}. * 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); super(segmentReaders);
this.directory = directory; this.directory = directory;
} }

View File

@ -227,11 +227,13 @@ 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(); assert indexWriter.holdsFullFlushLock();
if (infoStream.isEnabled("DW")) { if (infoStream.isEnabled("DW")) {
infoStream.message("DW", "lockAndAbortAll"); infoStream.message("DW", "lockAndAbortAll");
} }
long abortedDocCount = 0;
boolean success = false; boolean success = false;
try { try {
deleteQueue.clear(); deleteQueue.clear();
@ -239,12 +241,13 @@ final class DocumentsWriter implements Closeable, Accountable {
for (int i = 0; i < limit; i++) { for (int i = 0; i < limit; i++) {
final ThreadState perThread = perThreadPool.getThreadState(i); final ThreadState perThread = perThreadPool.getThreadState(i);
perThread.lock(); perThread.lock();
abortThreadState(perThread); abortedDocCount += abortThreadState(perThread);
} }
deleteQueue.clear(); deleteQueue.clear();
flushControl.abortPendingFlushes(); flushControl.abortPendingFlushes();
flushControl.waitForFlush(); flushControl.waitForFlush();
success = true; success = true;
return abortedDocCount;
} finally { } finally {
if (infoStream.isEnabled("DW")) { if (infoStream.isEnabled("DW")) {
infoStream.message("DW", "finished lockAndAbortAll success=" + success); infoStream.message("DW", "finished lockAndAbortAll success=" + success);
@ -256,21 +259,27 @@ 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(); assert perThread.isHeldByCurrentThread();
if (perThread.isActive()) { // we might be closed if (perThread.isActive()) { // we might be closed
if (perThread.isInitialized()) { if (perThread.isInitialized()) {
try { try {
subtractFlushedNumDocs(perThread.dwpt.getNumDocsInRAM()); int abortedDocCount = perThread.dwpt.getNumDocsInRAM();
subtractFlushedNumDocs(abortedDocCount);
perThread.dwpt.abort(); perThread.dwpt.abort();
return abortedDocCount;
} finally { } finally {
flushControl.doOnAbort(perThread); flushControl.doOnAbort(perThread);
} }
} else { } else {
flushControl.doOnAbort(perThread); flushControl.doOnAbort(perThread);
// This DWPT was never initialized so it has no indexed documents:
return 0;
} }
} else { } else {
assert closed; 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 /** Anything that will add N docs to the index should reserve first to
* make sure it's allowed. */ * make sure it's allowed. */
private void reserveDoc() { private void reserveOneDoc() {
if (pendingNumDocs.incrementAndGet() > IndexWriter.getActualMaxDocs()) { if (pendingNumDocs.incrementAndGet() > IndexWriter.getActualMaxDocs()) {
// Reserve failed // Reserve failed: put the one doc back and throw exc:
pendingNumDocs.decrementAndGet(); 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 { public void updateDocument(IndexDocument doc, Analyzer analyzer, Term delTerm) throws IOException, AbortingException {
testPoint("DocumentsWriterPerThread addDocument start"); testPoint("DocumentsWriterPerThread addDocument start");
assert deleteQueue != null; assert deleteQueue != null;
reserveOneDoc();
docState.doc = doc; docState.doc = doc;
docState.analyzer = analyzer; docState.analyzer = analyzer;
docState.docID = numDocsInRAM; docState.docID = numDocsInRAM;
@ -224,7 +225,6 @@ class DocumentsWriterPerThread {
// document, so the counter will be "wrong" in that case, but // document, so the counter will be "wrong" in that case, but
// it's very hard to fix (we can't easily distinguish aborting // it's very hard to fix (we can't easily distinguish aborting
// vs non-aborting exceptions): // vs non-aborting exceptions):
reserveDoc();
boolean success = false; boolean success = false;
try { try {
try { try {
@ -261,7 +261,7 @@ class DocumentsWriterPerThread {
// document, so the counter will be "wrong" in that case, but // document, so the counter will be "wrong" in that case, but
// it's very hard to fix (we can't easily distinguish aborting // it's very hard to fix (we can't easily distinguish aborting
// vs non-aborting exceptions): // vs non-aborting exceptions):
reserveDoc(); reserveOneDoc();
docState.doc = doc; docState.doc = doc;
docState.docID = numDocsInRAM; docState.docID = numDocsInRAM;
docCount++; 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 in DirectoryReader that this ExitableDirectoryReader wraps around to make it Exitable.
* @param queryTimeout The object to periodically check if the query should time out. * @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)); super(in, new ExitableSubReaderWrapper(queryTimeout));
this.queryTimeout = queryTimeout; this.queryTimeout = queryTimeout;
} }
@Override @Override
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) { protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
return new ExitableDirectoryReader(in, queryTimeout); 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)}) * can be used normally (e.g. passed to {@link DirectoryReader#openIfChanged(DirectoryReader)})
* and so on. * 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); return new ExitableDirectoryReader(in, queryTimeout);
} }

View File

@ -79,7 +79,7 @@ public abstract class FilterDirectoryReader extends DirectoryReader {
* @param in the DirectoryReader to filter * @param in the DirectoryReader to filter
* @param wrapper the SubReaderWrapper to use to wrap subreaders * @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())); super(in.directory(), wrapper.wrap(in.getSequentialSubReaders()));
this.in = in; this.in = in;
} }
@ -93,9 +93,9 @@ public abstract class FilterDirectoryReader extends DirectoryReader {
* @param in the DirectoryReader to wrap * @param in the DirectoryReader to wrap
* @return the wrapped DirectoryReader * @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); 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 { public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
/** Hard limit on maximum number of documents that may be added to the /** 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 // 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 on "typical" JVMs. We don't just use
// ArrayUtil.MAX_ARRAY_LENGTH here because this can vary across JVMs: // 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(); rollbackSegments = segmentInfos.createBackupSegmentInfos();
pendingNumDocs.set(segmentInfos.totalDocCount());
// start with previous field numbers, but new FieldInfos // start with previous field numbers, but new FieldInfos
globalFieldNumberMap = getFieldNumberMap(); globalFieldNumberMap = getFieldNumberMap();
@ -2055,13 +2056,16 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
*/ */
try { try {
synchronized (fullFlushLock) { synchronized (fullFlushLock) {
docWriter.lockAndAbortAll(this); long abortedDocCount = docWriter.lockAndAbortAll(this);
pendingNumDocs.addAndGet(-abortedDocCount);
processEvents(false, true); processEvents(false, true);
synchronized (this) { synchronized (this) {
try { try {
// Abort any running merges // Abort any running merges
abortMerges(); abortMerges();
// Remove all segments // Remove all segments
pendingNumDocs.addAndGet(-segmentInfos.totalDocCount());
segmentInfos.clear(); segmentInfos.clear();
// Ask deleter to locate unreferenced files & remove them: // Ask deleter to locate unreferenced files & remove them:
deleter.checkpoint(segmentInfos, false); deleter.checkpoint(segmentInfos, false);
@ -2077,6 +2081,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
++changeCount; ++changeCount;
segmentInfos.changed(); segmentInfos.changed();
globalFieldNumberMap.clear(); globalFieldNumberMap.clear();
success = true; success = true;
} finally { } finally {
docWriter.unlockAllAfterAbortAll(this); 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 IOException if there is a low-level IO error
* @throws LockObtainFailedException if we were unable to * @throws LockObtainFailedException if we were unable to
* acquire the write lock in at least one directory * 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 { public void addIndexes(Directory... dirs) throws IOException {
ensureOpen(); ensureOpen();
@ -2335,16 +2342,25 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
flush(false, true); flush(false, true);
List<SegmentCommitInfo> infos = new ArrayList<>(); List<SegmentCommitInfo> infos = new ArrayList<>();
int totalDocCount = 0;
boolean success = false; // long so we can detect int overflow:
try { long totalDocCount = 0;
List<SegmentInfos> commits = new ArrayList<>(dirs.length);
for (Directory dir : dirs) { for (Directory dir : dirs) {
if (infoStream.isEnabled("IW")) { if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "addIndexes: process directory " + dir); infoStream.message("IW", "addIndexes: process directory " + dir);
} }
SegmentInfos sis = SegmentInfos.readLatestCommit(dir); // read infos from dir SegmentInfos sis = SegmentInfos.readLatestCommit(dir); // read infos from dir
totalDocCount += sis.totalDocCount(); totalDocCount += sis.totalDocCount();
commits.add(sis);
}
// Best-effort up front check:
testReserveDocs(totalDocCount);
boolean success = false;
try {
for (SegmentInfos sis : commits) {
for (SegmentCommitInfo info : sis) { for (SegmentCommitInfo info : sis) {
assert !infos.contains(info): "dup info dir=" + info.info.dir + " name=" + info.info.name; 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 { } finally {
if (!success) { if (!success) {
for(SegmentCommitInfo sipc : infos) { for(SegmentCommitInfo sipc : infos) {
for(String file : sipc.files()) { IOUtils.deleteFilesIgnoringExceptions(directory, sipc.files().toArray(new String[0]));
try {
directory.deleteFile(file);
} catch (Throwable t) {
}
}
} }
} }
} }
@ -2381,9 +2392,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
success = false; success = false;
try { try {
ensureOpen(); 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); reserveDocs(totalDocCount);
success = true; success = true;
} finally { } finally {
if (!success) { if (!success) {
@ -2441,10 +2453,14 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
* if the index is corrupt * if the index is corrupt
* @throws IOException * @throws IOException
* if there is a low-level IO error * 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 { public void addIndexes(CodecReader... readers) throws IOException {
ensureOpen(); ensureOpen();
int numDocs = 0;
// long so we can detect int overflow:
long numDocs = 0;
try { try {
if (infoStream.isEnabled("IW")) { if (infoStream.isEnabled("IW")) {
@ -2457,11 +2473,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
numDocs += leaf.numDocs(); numDocs += leaf.numDocs();
} }
// Make sure adding the new documents to this index won't // Best-effort up front check:
// exceed the limit: testReserveDocs(numDocs);
reserveDocs(numDocs);
final IOContext context = new IOContext(new MergeInfo(numDocs, -1, false, -1)); final IOContext context = new IOContext(new MergeInfo(Math.toIntExact(numDocs), -1, false, -1));
// TODO: somehow we should fix this merge so it's // TODO: somehow we should fix this merge so it's
// abortable so that IW.close(false) is able to stop it // abortable so that IW.close(false) is able to stop it
@ -2553,11 +2568,15 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
return; return;
} }
ensureOpen(); ensureOpen();
// Now reserve the docs, just before we update SIS:
reserveDocs(numDocs);
segmentInfos.add(infoPerCommit); segmentInfos.add(infoPerCommit);
checkpoint(); checkpoint();
} }
} catch (OutOfMemoryError oom) { } catch (OutOfMemoryError oom) {
tragicEvent(oom, "addIndexes(IndexReader...)"); tragicEvent(oom, "addIndexes(CodecReader...)");
} }
maybeMerge(); 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 /** Anything that will add N docs to the index should reserve first to
* make sure it's allowed. This will throw {@code * make sure it's allowed. This will throw {@code
* IllegalStateException} if it's not allowed. */ * IllegalArgumentException} if it's not allowed. */
private void reserveDocs(int numDocs) { private void reserveDocs(long addedNumDocs) {
if (pendingNumDocs.addAndGet(numDocs) > actualMaxDocs) { assert addedNumDocs >= 0;
// Reserve failed if (pendingNumDocs.addAndGet(addedNumDocs) > actualMaxDocs) {
pendingNumDocs.addAndGet(-numDocs); // Reserve failed: put the docs back and throw exc:
throw new IllegalStateException("number of documents in the index cannot exceed " + actualMaxDocs); 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 /** Wraps the incoming {@link Directory} so that we assign a per-thread
* {@link MergeRateLimiter} to all created {@link IndexOutput}s. */ * {@link MergeRateLimiter} to all created {@link IndexOutput}s. */
private Directory addMergeRateLimiters(Directory in) { 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> * <p>Note that all subreaders are closed if this Multireader is closed.</p>
* @param subReaders set of (sub)readers * @param subReaders set of (sub)readers
*/ */
public MultiReader(IndexReader... subReaders) { public MultiReader(IndexReader... subReaders) throws IOException {
this(subReaders, true); this(subReaders, true);
} }
@ -55,7 +55,7 @@ public class MultiReader extends BaseCompositeReader<IndexReader> {
* @param closeSubReaders indicates whether the subreaders should be closed * @param closeSubReaders indicates whether the subreaders should be closed
* when this MultiReader is closed * when this MultiReader is closed
*/ */
public MultiReader(IndexReader[] subReaders, boolean closeSubReaders) { public MultiReader(IndexReader[] subReaders, boolean closeSubReaders) throws IOException {
super(subReaders.clone()); super(subReaders.clone());
this.closeSubReaders = closeSubReaders; this.closeSubReaders = closeSubReaders;
if (!closeSubReaders) { if (!closeSubReaders) {

View File

@ -284,6 +284,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
if (numSegments < 0) { if (numSegments < 0) {
throw new CorruptIndexException("invalid segment count: " + numSegments, input); throw new CorruptIndexException("invalid segment count: " + numSegments, input);
} }
long totalDocs = 0;
for (int seg = 0; seg < numSegments; seg++) { for (int seg = 0; seg < numSegments; seg++) {
String segName = input.readString(); String segName = input.readString();
final byte segmentID[]; final byte segmentID[];
@ -297,6 +298,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
Codec codec = Codec.forName(input.readString()); Codec codec = Codec.forName(input.readString());
SegmentInfo info = codec.segmentInfoFormat().read(directory, segName, segmentID, IOContext.READ); SegmentInfo info = codec.segmentInfoFormat().read(directory, segName, segmentID, IOContext.READ);
info.setCodec(codec); info.setCodec(codec);
totalDocs += info.getDocCount();
long delGen = input.readLong(); long delGen = input.readLong();
int delCount = input.readInt(); int delCount = input.readInt();
if (delCount < 0 || delCount > info.getDocCount()) { if (delCount < 0 || delCount > info.getDocCount()) {
@ -323,6 +325,11 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
CodecUtil.checkFooter(input); 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; return infos;
} }
} }
@ -722,11 +729,13 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
/** Returns sum of all segment's docCounts. Note that /** Returns sum of all segment's docCounts. Note that
* this does not include deletions */ * this does not include deletions */
public int totalDocCount() { public int totalDocCount() {
int count = 0; long count = 0;
for(SegmentCommitInfo info : this) { for(SegmentCommitInfo info : this) {
count += info.info.getDocCount(); 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 /** 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 */ /** called only from static open() methods */
StandardDirectoryReader(Directory directory, LeafReader[] readers, IndexWriter writer, StandardDirectoryReader(Directory directory, LeafReader[] readers, IndexWriter writer,
SegmentInfos sis, boolean applyAllDeletes) { SegmentInfos sis, boolean applyAllDeletes) throws IOException {
super(directory, readers); super(directory, readers);
this.writer = writer; this.writer = writer;
this.segmentInfos = sis; this.segmentInfos = sis;
@ -52,19 +52,24 @@ final class StandardDirectoryReader extends DirectoryReader {
protected DirectoryReader doBody(String segmentFileName) throws IOException { protected DirectoryReader doBody(String segmentFileName) throws IOException {
SegmentInfos sis = SegmentInfos.readCommit(directory, segmentFileName); SegmentInfos sis = SegmentInfos.readCommit(directory, segmentFileName);
final SegmentReader[] readers = new SegmentReader[sis.size()]; final SegmentReader[] readers = new SegmentReader[sis.size()];
for (int i = sis.size()-1; i >= 0; i--) {
boolean success = false; boolean success = false;
try { try {
for (int i = sis.size()-1; i >= 0; i--) {
readers[i] = new SegmentReader(sis.info(i), IOContext.READ); readers[i] = new SegmentReader(sis.info(i), IOContext.READ);
}
// 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; success = true;
return reader;
} finally { } finally {
if (!success) { if (success == false) {
IOUtils.closeWhileHandlingException(readers); IOUtils.closeWhileHandlingException(readers);
} }
} }
} }
return new StandardDirectoryReader(directory, readers, null, sis, false);
}
}.run(commit); }.run(commit);
} }

View File

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

View File

@ -17,8 +17,10 @@ package org.apache.lucene.index;
* limitations under the License. * limitations under the License.
*/ */
import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.concurrent.CountDownLatch;
import org.apache.lucene.document.Document; import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field; 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.TermQuery;
import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.TopDocs;
import org.apache.lucene.store.Directory; 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.SuppressCodecs;
import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil; import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.TimeUnits; import org.apache.lucene.util.TimeUnits;
import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite; import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
@SuppressCodecs({ "SimpleText", "Memory", "Direct" }) @SuppressCodecs({ "SimpleText", "Memory", "Direct" })
@ -93,7 +97,7 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
try { try {
w.addDocument(new Document()); w.addDocument(new Document());
fail("didn't hit exception"); fail("didn't hit exception");
} catch (IllegalStateException ise) { } catch (IllegalArgumentException iae) {
// expected // expected
} }
w.close(); w.close();
@ -116,7 +120,7 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
try { try {
w.addDocuments(Collections.singletonList(new Document())); w.addDocuments(Collections.singletonList(new Document()));
fail("didn't hit exception"); fail("didn't hit exception");
} catch (IllegalStateException ise) { } catch (IllegalArgumentException iae) {
// expected // expected
} }
w.close(); w.close();
@ -139,7 +143,7 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
try { try {
w.updateDocument(new Term("field", "foo"), new Document()); w.updateDocument(new Term("field", "foo"), new Document());
fail("didn't hit exception"); fail("didn't hit exception");
} catch (IllegalStateException ise) { } catch (IllegalArgumentException iae) {
// expected // expected
} }
w.close(); w.close();
@ -162,7 +166,7 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
try { try {
w.updateDocuments(new Term("field", "foo"), Collections.singletonList(new Document())); w.updateDocuments(new Term("field", "foo"), Collections.singletonList(new Document()));
fail("didn't hit exception"); fail("didn't hit exception");
} catch (IllegalStateException ise) { } catch (IllegalArgumentException iae) {
// expected // expected
} }
w.close(); w.close();
@ -201,7 +205,7 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
try { try {
w.addDocument(new Document()); w.addDocument(new Document());
fail("didn't hit exception"); fail("didn't hit exception");
} catch (IllegalStateException ise) { } catch (IllegalArgumentException iae) {
// expected // expected
} }
w.close(); w.close();
@ -247,7 +251,7 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
try { try {
w.addDocument(new Document()); w.addDocument(new Document());
fail("didn't hit exception"); fail("didn't hit exception");
} catch (IllegalStateException ise) { } catch (IllegalArgumentException iae) {
// expected // expected
} }
w.close(); w.close();
@ -273,7 +277,7 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
try { try {
w2.addIndexes(new Directory[] {dir}); w2.addIndexes(new Directory[] {dir});
fail("didn't hit exception"); fail("didn't hit exception");
} catch (IllegalStateException ise) { } catch (IllegalArgumentException iae) {
// expected // expected
} }
assertEquals(1, w2.maxDoc()); assertEquals(1, w2.maxDoc());
@ -281,7 +285,7 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
try { try {
TestUtil.addIndexesSlowly(w2, ir); TestUtil.addIndexesSlowly(w2, ir);
fail("didn't hit exception"); fail("didn't hit exception");
} catch (IllegalStateException ise) { } catch (IllegalArgumentException iae) {
// expected // expected
} }
w2.close(); w2.close();
@ -370,6 +374,103 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
dir2.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 { public void testTooLargeMaxDocs() throws Exception {
try { try {
IndexWriter.setMaxDocs(Integer.MAX_VALUE); IndexWriter.setMaxDocs(Integer.MAX_VALUE);
@ -378,4 +479,244 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
// expected // 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 { private static class MyFilterDirectoryReader extends FilterDirectoryReader {
public MyFilterDirectoryReader(DirectoryReader in) { public MyFilterDirectoryReader(DirectoryReader in) throws IOException {
super(in, super(in,
new FilterDirectoryReader.SubReaderWrapper() { new FilterDirectoryReader.SubReaderWrapper() {
@Override @Override
@ -470,7 +470,7 @@ public class TestSearcherManager extends ThreadedIndexingAndSearchingTestCase {
} }
@Override @Override
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) { protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
return new MyFilterDirectoryReader(in); return new MyFilterDirectoryReader(in);
} }
} }

View File

@ -19,8 +19,10 @@ package org.apache.lucene.store;
import java.io.IOException; import java.io.IOException;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.LuceneTestCase;
@ -91,4 +93,16 @@ public class TestMockDirectoryWrapper extends LuceneTestCase {
dir.close(); 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 java.util.TreeSet;
import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexReader; 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.LeafReader;
import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.MultiReader; import org.apache.lucene.index.MultiReader;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.ReaderUtil; import org.apache.lucene.index.ReaderUtil;
import org.apache.lucene.index.StoredFieldVisitor; import org.apache.lucene.index.StoredFieldVisitor;
import org.apache.lucene.index.Term; import org.apache.lucene.index.Term;
@ -100,7 +100,14 @@ public class PostingsHighlighter {
// unnecessary. // unnecessary.
/** for rewriting: we don't want slow processing from MTQs */ /** 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 /** Default maximum content size to process. Typically snippets
* closer to the beginning of the document better summarize its content */ * 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)}) * can be used normally (e.g. passed to {@link DirectoryReader#openIfChanged(DirectoryReader)})
* and so on. * 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); return new UninvertingDirectoryReader(in, mapping);
} }
static class UninvertingDirectoryReader extends FilterDirectoryReader { static class UninvertingDirectoryReader extends FilterDirectoryReader {
final Map<String,Type> mapping; 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() { super(in, new FilterDirectoryReader.SubReaderWrapper() {
@Override @Override
public LeafReader wrap(LeafReader reader) { public LeafReader wrap(LeafReader reader) {
@ -164,7 +164,7 @@ public class UninvertingReader extends FilterLeafReader {
} }
@Override @Override
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) { protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
return new UninvertingDirectoryReader(in, mapping); return new UninvertingDirectoryReader(in, mapping);
} }
} }

View File

@ -17,6 +17,8 @@ package org.apache.lucene.index;
* limitations under the License. * limitations under the License.
*/ */
import java.io.IOException;
/** /**
* A {@link DirectoryReader} that wraps all its subreaders with * A {@link DirectoryReader} that wraps all its subreaders with
* {@link AssertingLeafReader} * {@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()); super(in, new AssertingSubReaderWrapper());
} }
@Override @Override
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) { protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
return new AssertingDirectoryReader(in); return new AssertingDirectoryReader(in);
} }

View File

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

View File

@ -17,6 +17,7 @@ package org.apache.lucene.index;
* limitations under the License. * limitations under the License.
*/ */
import java.io.IOException;
import java.util.Random; 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)); super(in, new MismatchedSubReaderWrapper(random));
} }
@Override @Override
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) { protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
return new AssertingDirectoryReader(in); return new AssertingDirectoryReader(in);
} }
} }

View File

@ -22,6 +22,7 @@ import java.util.List;
import java.util.Random; import java.util.Random;
import junit.framework.Assert; import junit.framework.Assert;
import org.apache.lucene.analysis.MockAnalyzer; import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document; import org.apache.lucene.document.Document;
import org.apache.lucene.index.AllDeletedFilterReader; import org.apache.lucene.index.AllDeletedFilterReader;
@ -133,7 +134,7 @@ public class QueryUtils {
public static class FCInvisibleMultiReader extends MultiReader { public static class FCInvisibleMultiReader extends MultiReader {
private final Object cacheKey = new Object(); private final Object cacheKey = new Object();
public FCInvisibleMultiReader(IndexReader... readers) { public FCInvisibleMultiReader(IndexReader... readers) throws IOException {
super(readers); 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 make any methods in this class synchronized, volatile
// do NOT import anything from the concurrency package. // do NOT import anything from the concurrency package.
// no randoms, no nothing. // no randoms, no nothing.
public class BaseDirectoryWrapper extends FilterDirectory { public abstract class BaseDirectoryWrapper extends FilterDirectory {
private boolean checkIndexOnClose = true; private boolean checkIndexOnClose = true;
private boolean crossCheckTermVectorsOnClose = true; private boolean crossCheckTermVectorsOnClose = true;
protected volatile boolean isOpen = true; protected volatile boolean isOpen = true;
public BaseDirectoryWrapper(Directory delegate) { protected BaseDirectoryWrapper(Directory delegate) {
super(delegate); super(delegate);
} }
@ -72,10 +72,4 @@ public class BaseDirectoryWrapper extends FilterDirectory {
public boolean getCrossCheckTermVectorsOnClose() { public boolean getCrossCheckTermVectorsOnClose() {
return crossCheckTermVectorsOnClose; 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; 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 @Override
public synchronized void sync(Collection<String> names) throws IOException { public synchronized void sync(Collection<String> names) throws IOException {
maybeYield(); maybeYield();
@ -233,17 +216,14 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
if (crashed) { if (crashed) {
throw new IOException("cannot sync after crash"); throw new IOException("cannot sync after crash");
} }
// don't wear out our hardware so much in tests. // always pass thru fsync, directories rely on this.
if (LuceneTestCase.rarely(randomState) || mustSync()) { // 90% of time, we use DisableFsyncFS which omits the real calls.
for (String name : names) { for (String name : names) {
// randomly fail with IOE on any file // randomly fail with IOE on any file
maybeThrowIOException(name); maybeThrowIOException(name);
in.sync(Collections.singleton(name)); in.sync(Collections.singleton(name));
unSyncedFiles.remove(name); unSyncedFiles.remove(name);
} }
} else {
unSyncedFiles.removeAll(names);
}
} }
@Override @Override
@ -1053,18 +1033,36 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
} }
} }
// 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}, /** Use this when throwing fake {@code IOException},
* e.g. from {@link MockDirectoryWrapper.Failure}. */ * e.g. from {@link MockDirectoryWrapper.Failure}. */
public static class FakeIOException extends IOException { 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.NoInstanceHooksOverridesRule;
import com.carrotsearch.randomizedtesting.rules.StaticFieldsInvariantRule; import com.carrotsearch.randomizedtesting.rules.StaticFieldsInvariantRule;
import com.carrotsearch.randomizedtesting.rules.SystemPropertiesInvariantRule; import com.carrotsearch.randomizedtesting.rules.SystemPropertiesInvariantRule;
import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.MockAnalyzer; import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Field; 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;
import org.apache.lucene.store.MockDirectoryWrapper.Throttling; import org.apache.lucene.store.MockDirectoryWrapper.Throttling;
import org.apache.lucene.store.NRTCachingDirectory; 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.AutomatonTestUtil;
import org.apache.lucene.util.automaton.CompiledAutomaton; import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.automaton.RegExp; import org.apache.lucene.util.automaton.RegExp;
@ -1317,7 +1319,7 @@ public abstract class LuceneTestCase extends Assert {
} }
if (bare) { if (bare) {
BaseDirectoryWrapper base = new BaseDirectoryWrapper(directory); BaseDirectoryWrapper base = new RawDirectoryWrapper(directory);
closeAfterSuite(new CloseableDirectory(base, suiteFailureMarker)); closeAfterSuite(new CloseableDirectory(base, suiteFailureMarker));
return base; return base;
} else { } else {

View File

@ -67,13 +67,13 @@ import org.apache.lucene.uninverting.UninvertingReader;
import org.apache.lucene.util.Bits; import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet; 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.ErrorCode;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap; 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.DirContext;
import org.apache.solr.core.DirectoryFactory;
import org.apache.solr.core.SolrConfig; import org.apache.solr.core.SolrConfig;
import org.apache.solr.core.SolrCore; import org.apache.solr.core.SolrCore;
import org.apache.solr.core.SolrInfoMBean; 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() ? // TODO: wrap elsewhere and return a "map" from the schema that overrides get() ?
// this reader supports reopen // 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; assert reader != null;
return ExitableDirectoryReader.wrap return ExitableDirectoryReader.wrap
(UninvertingReader.wrap(reader, core.getLatestSchema().getUninversionMap(reader)), (UninvertingReader.wrap(reader, core.getLatestSchema().getUninversionMap(reader)),