blob: 1278eafa590ea4187172a9989794b9422f9acc55 [file] [log] [blame]
Index: lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java (revision 1662487)
+++ lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java (working copy)
@@ -64,24 +64,32 @@
* 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
Index: lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java (revision 1662487)
+++ lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java (working copy)
@@ -305,11 +305,11 @@
* 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
Index: lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (revision 1662487)
+++ lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (working copy)
@@ -205,7 +205,7 @@
if (pendingNumDocs.incrementAndGet() > IndexWriter.getActualMaxDocs()) {
// Reserve failed
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());
}
}
@@ -212,6 +212,7 @@
public void updateDocument(IndexDocument doc, Analyzer analyzer, Term delTerm) throws IOException, AbortingException {
testPoint("DocumentsWriterPerThread addDocument start");
assert deleteQueue != null;
+ reserveDoc();
docState.doc = doc;
docState.analyzer = analyzer;
docState.docID = numDocsInRAM;
@@ -224,7 +225,6 @@
// 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 {
Index: lucene/core/src/java/org/apache/lucene/index/ExitableDirectoryReader.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/index/ExitableDirectoryReader.java (revision 1662487)
+++ lucene/core/src/java/org/apache/lucene/index/ExitableDirectoryReader.java (working copy)
@@ -185,13 +185,13 @@
* @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 @@
* 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);
}
Index: lucene/core/src/java/org/apache/lucene/index/FilterDirectoryReader.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/index/FilterDirectoryReader.java (revision 1662487)
+++ lucene/core/src/java/org/apache/lucene/index/FilterDirectoryReader.java (working copy)
@@ -79,7 +79,7 @@
* @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 @@
* @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);
}
Index: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (revision 1662487)
+++ lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (working copy)
@@ -196,7 +196,7 @@
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 @@
}
rollbackSegments = segmentInfos.createBackupSegmentInfos();
+ pendingNumDocs.set(segmentInfos.totalDocCount());
// start with previous field numbers, but new FieldInfos
globalFieldNumberMap = getFieldNumberMap();
@@ -2077,6 +2078,7 @@
++changeCount;
segmentInfos.changed();
globalFieldNumberMap.clear();
+ pendingNumDocs.set(0);
success = true;
} finally {
docWriter.unlockAllAfterAbortAll(this);
@@ -2317,6 +2319,8 @@
* @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 +2339,26 @@
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);
+ }
+
+ // Make sure adding the new documents to this index won't
+ // exceed the limit:
+ reserveDocs(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 +2381,7 @@
} 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 +2390,6 @@
success = false;
try {
ensureOpen();
- // Make sure adding the new documents to this index won't
- // exceed the limit:
- reserveDocs(totalDocCount);
success = true;
} finally {
if (!success) {
@@ -2441,11 +2447,15 @@
* 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")) {
infoStream.message("IW", "flush at addIndexes(CodecReader...)");
@@ -2461,7 +2471,7 @@
// exceed the limit:
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
// abortable so that IW.close(false) is able to stop it
@@ -2557,7 +2567,7 @@
checkpoint();
}
} catch (OutOfMemoryError oom) {
- tragicEvent(oom, "addIndexes(IndexReader...)");
+ tragicEvent(oom, "addIndexes(CodecReader...)");
}
maybeMerge();
}
@@ -4616,12 +4626,12 @@
/** 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) {
+ * IllegalArgumentException} if it's not allowed. */
+ private void reserveDocs(long numDocs) {
if (pendingNumDocs.addAndGet(numDocs) > actualMaxDocs) {
// Reserve failed
pendingNumDocs.addAndGet(-numDocs);
- throw new IllegalStateException("number of documents in the index cannot exceed " + actualMaxDocs);
+ throw new IllegalArgumentException("number of documents in the index cannot exceed " + actualMaxDocs + " (current document count is " + pendingNumDocs.get() + "; added numDocs is " + numDocs + ")");
}
}
Index: lucene/core/src/java/org/apache/lucene/index/MultiReader.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/index/MultiReader.java (revision 1662487)
+++ lucene/core/src/java/org/apache/lucene/index/MultiReader.java (working copy)
@@ -45,7 +45,7 @@
* <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 @@
* @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) {
Index: lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java (revision 1662487)
+++ lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java (working copy)
@@ -284,6 +284,7 @@
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 @@
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 @@
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 @@
/** 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
Index: lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java (revision 1662487)
+++ lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java (working copy)
@@ -38,7 +38,7 @@
/** 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 @@
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);
}
Index: lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java
===================================================================
--- lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java (revision 1662487)
+++ lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java (working copy)
@@ -191,7 +191,7 @@
}
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 @@
}
@Override
- protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) {
+ protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
return new ParallelLeafDirectoryReader(in);
}
Index: lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java
===================================================================
--- lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java (revision 1662487)
+++ lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java (working copy)
@@ -17,6 +17,7 @@
* limitations under the License.
*/
+import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
@@ -28,6 +29,9 @@
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;
@@ -93,7 +97,7 @@
try {
w.addDocument(new Document());
fail("didn't hit exception");
- } catch (IllegalStateException ise) {
+ } catch (IllegalArgumentException iae) {
// expected
}
w.close();
@@ -116,7 +120,7 @@
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 @@
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 @@
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 @@
try {
w.addDocument(new Document());
fail("didn't hit exception");
- } catch (IllegalStateException ise) {
+ } catch (IllegalArgumentException iae) {
// expected
}
w.close();
@@ -247,7 +251,7 @@
try {
w.addDocument(new Document());
fail("didn't hit exception");
- } catch (IllegalStateException ise) {
+ } catch (IllegalArgumentException iae) {
// expected
}
w.close();
@@ -273,7 +277,7 @@
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 @@
try {
TestUtil.addIndexesSlowly(w2, ir);
fail("didn't hit exception");
- } catch (IllegalStateException ise) {
+ } catch (IllegalArgumentException iae) {
// expected
}
w2.close();
@@ -369,7 +373,104 @@
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 {
IndexWriter.setMaxDocs(Integer.MAX_VALUE);
@@ -378,4 +479,90 @@
// 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());
+ 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();
+ }
}
Index: lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java
===================================================================
--- lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java (revision 1662487)
+++ lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java (working copy)
@@ -457,7 +457,7 @@
}
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 @@
}
@Override
- protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) {
+ protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
return new MyFilterDirectoryReader(in);
}
}
Index: lucene/core/src/test/org/apache/lucene/store/TestMockDirectoryWrapper.java
===================================================================
--- lucene/core/src/test/org/apache/lucene/store/TestMockDirectoryWrapper.java (revision 1662487)
+++ lucene/core/src/test/org/apache/lucene/store/TestMockDirectoryWrapper.java (working copy)
@@ -19,8 +19,10 @@
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 @@
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();
+ }
+
}
Index: lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/PostingsHighlighter.java
===================================================================
--- lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/PostingsHighlighter.java (revision 1662487)
+++ lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/PostingsHighlighter.java (working copy)
@@ -31,7 +31,6 @@
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.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 @@
// 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 */
Index: lucene/misc/src/java/org/apache/lucene/uninverting/UninvertingReader.java
===================================================================
--- lucene/misc/src/java/org/apache/lucene/uninverting/UninvertingReader.java (revision 1662487)
+++ lucene/misc/src/java/org/apache/lucene/uninverting/UninvertingReader.java (working copy)
@@ -146,7 +146,7 @@
* 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);
}
@@ -153,7 +153,7 @@
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 @@
}
@Override
- protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) {
+ protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
return new UninvertingDirectoryReader(in, mapping);
}
}
Index: lucene/test-framework/src/java/org/apache/lucene/index/AssertingDirectoryReader.java
===================================================================
--- lucene/test-framework/src/java/org/apache/lucene/index/AssertingDirectoryReader.java (revision 1662487)
+++ lucene/test-framework/src/java/org/apache/lucene/index/AssertingDirectoryReader.java (working copy)
@@ -17,6 +17,8 @@
* limitations under the License.
*/
+import java.io.IOException;
+
/**
* A {@link DirectoryReader} that wraps all its subreaders with
* {@link AssertingLeafReader}
@@ -30,12 +32,12 @@
}
}
- 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);
}
Index: lucene/test-framework/src/java/org/apache/lucene/index/BaseStoredFieldsFormatTestCase.java
===================================================================
--- lucene/test-framework/src/java/org/apache/lucene/index/BaseStoredFieldsFormatTestCase.java (revision 1662487)
+++ lucene/test-framework/src/java/org/apache/lucene/index/BaseStoredFieldsFormatTestCase.java (working copy)
@@ -599,7 +599,7 @@
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 @@
}
@Override
- protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) {
+ protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
return new DummyFilterDirectoryReader(in);
}
Index: lucene/test-framework/src/java/org/apache/lucene/index/MismatchedDirectoryReader.java
===================================================================
--- lucene/test-framework/src/java/org/apache/lucene/index/MismatchedDirectoryReader.java (revision 1662487)
+++ lucene/test-framework/src/java/org/apache/lucene/index/MismatchedDirectoryReader.java (working copy)
@@ -17,6 +17,7 @@
* limitations under the License.
*/
+import java.io.IOException;
import java.util.Random;
/**
@@ -38,12 +39,12 @@
}
}
- 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);
}
}
Index: lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
===================================================================
--- lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java (revision 1662487)
+++ lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java (working copy)
@@ -22,6 +22,7 @@
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 static class FCInvisibleMultiReader extends MultiReader {
private final Object cacheKey = new Object();
- public FCInvisibleMultiReader(IndexReader... readers) {
+ public FCInvisibleMultiReader(IndexReader... readers) throws IOException {
super(readers);
}
Index: lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryWrapper.java
===================================================================
--- lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryWrapper.java (revision 1662487)
+++ lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryWrapper.java (working copy)
@@ -28,13 +28,13 @@
// 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 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);
- }
}
Index: lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
===================================================================
--- lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java (revision 1662487)
+++ lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java (working copy)
@@ -209,23 +209,6 @@
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 @@
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 boolean isLocked() throws IOException {
return delegateLock.isLocked();
}
+ }
+
+ /** Use this when throwing fake {@code IOException},
+ * e.g. from {@link MockDirectoryWrapper.Failure}. */
+ public static class FakeIOException extends IOException {
}
- // 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);
+ 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:
- /** Use this when throwing fake {@code IOException},
- * e.g. from {@link MockDirectoryWrapper.Failure}. */
- public static class FakeIOException extends IOException {
+ @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();
+ }
}
Index: lucene/test-framework/src/java/org/apache/lucene/store/RawDirectoryWrapper.java
===================================================================
--- lucene/test-framework/src/java/org/apache/lucene/store/RawDirectoryWrapper.java (revision 0)
+++ lucene/test-framework/src/java/org/apache/lucene/store/RawDirectoryWrapper.java (working copy)
@@ -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();
+ }
+}
Property changes on: lucene/test-framework/src/java/org/apache/lucene/store/RawDirectoryWrapper.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
===================================================================
--- lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (revision 1662487)
+++ lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (working copy)
@@ -82,6 +82,7 @@
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.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 @@
}
if (bare) {
- BaseDirectoryWrapper base = new BaseDirectoryWrapper(directory);
+ BaseDirectoryWrapper base = new RawDirectoryWrapper(directory);
closeAfterSuite(new CloseableDirectory(base, suiteFailureMarker));
return base;
} else {
Index: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
===================================================================
--- solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java (revision 1662487)
+++ solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java (working copy)
@@ -67,13 +67,13 @@
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet;
+import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrException.ErrorCode;
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.DirContext;
import org.apache.solr.core.DirectoryFactory;
-import org.apache.solr.core.DirectoryFactory.DirContext;
import org.apache.solr.core.SolrConfig;
import org.apache.solr.core.SolrCore;
import org.apache.solr.core.SolrInfoMBean;
@@ -167,7 +167,7 @@
// 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)),