You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2015/02/27 11:20:24 UTC

svn commit: r1662654 - in /lucene/dev/branches/lucene_solr_4_10/lucene: ./ core/src/java/org/apache/lucene/index/ core/src/test/org/apache/lucene/index/

Author: mikemccand
Date: Fri Feb 27 10:20:24 2015
New Revision: 1662654

URL: http://svn.apache.org/r1662654
Log:
LUCENE-6299: IndexWriter was failing to enforce the 2.1 billion doc limit

Modified:
    lucene/dev/branches/lucene_solr_4_10/lucene/CHANGES.txt
    lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java
    lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java
    lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
    lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
    lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
    lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
    lucene/dev/branches/lucene_solr_4_10/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java

Modified: lucene/dev/branches/lucene_solr_4_10/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_10/lucene/CHANGES.txt?rev=1662654&r1=1662653&r2=1662654&view=diff
==============================================================================
--- lucene/dev/branches/lucene_solr_4_10/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/lucene_solr_4_10/lucene/CHANGES.txt Fri Feb 27 10:20:24 2015
@@ -809,6 +809,9 @@ New Features
   and MacOSX are known to work. On Windows, fsyncing a directory is not
   possible with Java APIs.  (Mike McCandless, Uwe Schindler)
 
+* LUCENE-6299: IndexWriter was failing to enforce the 2.1 billion doc
+  limit.  (Robert Muir, Mike McCandless)
+
 API Changes
 
 * LUCENE-5454: Add RandomAccessOrds, an optional extension of SortedSetDocValues

Modified: lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java?rev=1662654&r1=1662653&r2=1662654&view=diff
==============================================================================
--- lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java (original)
+++ lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java Fri Feb 27 10:20:24 2015
@@ -68,20 +68,22 @@ public abstract class BaseCompositeReade
     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()) {
+      throw new IllegalArgumentException("Too many documents: composite IndexReaders cannot exceed " + IndexWriter.getActualMaxDocs() + " but readers have total maxDoc=" + maxDoc);
+    }
+
+    starts[subReaders.length] = (int) maxDoc;
+    this.maxDoc = (int) maxDoc;
+    this.numDocs = (int) numDocs;
   }
 
   @Override

Modified: lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=1662654&r1=1662653&r2=1662654&view=diff
==============================================================================
--- lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java Fri Feb 27 10:20:24 2015
@@ -229,12 +229,14 @@ final class DocumentsWriter implements C
       }
     }
   }
-  
-  synchronized void lockAndAbortAll(IndexWriter indexWriter) {
+
+  /** Returns how many documents were aborted. */
+  synchronized long lockAndAbortAll(IndexWriter indexWriter) {
     assert indexWriter.holdsFullFlushLock();
     if (infoStream.isEnabled("DW")) {
       infoStream.message("DW", "lockAndAbortAll");
     }
+    long abortedDocCount = 0;
     boolean success = false;
     try {
       deleteQueue.clear();
@@ -243,13 +245,14 @@ final class DocumentsWriter implements C
       for (int i = 0; i < limit; i++) {
         final ThreadState perThread = perThreadPool.getThreadState(i);
         perThread.lock();
-        abortThreadState(perThread, newFilesSet);
+        abortedDocCount += abortThreadState(perThread, newFilesSet);
       }
       deleteQueue.clear();
       flushControl.abortPendingFlushes(newFilesSet);
       putEvent(new DeleteNewFilesEvent(newFilesSet));
       flushControl.waitForFlush();
       success = true;
+      return abortedDocCount;
     } finally {
       if (infoStream.isEnabled("DW")) {
         infoStream.message("DW", "finished lockAndAbortAll success=" + success);
@@ -261,22 +264,27 @@ final class DocumentsWriter implements C
     }
   }
 
-  private final void abortThreadState(final ThreadState perThread, Set<String> newFiles) {
+  private final int abortThreadState(final ThreadState perThread, Set<String> newFiles) {
     assert perThread.isHeldByCurrentThread();
     if (perThread.isActive()) { // we might be closed
       if (perThread.isInitialized()) { 
         try {
-          subtractFlushedNumDocs(perThread.dwpt.getNumDocsInRAM());
+          int abortedDocCount = perThread.dwpt.getNumDocsInRAM();
+          subtractFlushedNumDocs(abortedDocCount);
           perThread.dwpt.abort(newFiles);
+          return abortedDocCount;
         } finally {
           perThread.dwpt.checkAndResetHasAborted();
           flushControl.doOnAbort(perThread);
         }
       } else {
         flushControl.doOnAbort(perThread);
+        // This DWPT was never initialized so it has no indexed documents:
+        return 0;
       }
     } else {
       assert closed;
+      return 0;
     }
   }
   

Modified: lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1662654&r1=1662653&r2=1662654&view=diff
==============================================================================
--- lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Fri Feb 27 10:20:24 2015
@@ -211,17 +211,18 @@ class DocumentsWriterPerThread {
 
   /** Anything that will add N docs to the index should reserve first to
    *  make sure it's allowed. */
-  private void reserveDoc() {
+  private void reserveOneDoc() {
     if (pendingNumDocs.incrementAndGet() > IndexWriter.getActualMaxDocs()) {
-      // Reserve failed
+      // Reserve failed: put the one doc back and throw exc:
       pendingNumDocs.decrementAndGet();
-      throw new IllegalStateException("number of documents in the index cannot exceed " + IndexWriter.getActualMaxDocs());
+      throw new IllegalArgumentException("number of documents in the index cannot exceed " + IndexWriter.getActualMaxDocs());
     }
   }
 
   public void updateDocument(Iterable<? extends IndexableField> doc, Analyzer analyzer, Term delTerm) throws IOException {
     assert testPoint("DocumentsWriterPerThread addDocument start");
     assert deleteQueue != null;
+    reserveOneDoc();
     docState.doc = doc;
     docState.analyzer = analyzer;
     docState.docID = numDocsInRAM;
@@ -234,7 +235,6 @@ class DocumentsWriterPerThread {
     // document, so the counter will be "wrong" in that case, but
     // it's very hard to fix (we can't easily distinguish aborting
     // vs non-aborting exceptions):
-    reserveDoc();
     boolean success = false;
     try {
       try {
@@ -274,7 +274,7 @@ class DocumentsWriterPerThread {
         // document, so the counter will be "wrong" in that case, but
         // it's very hard to fix (we can't easily distinguish aborting
         // vs non-aborting exceptions):
-        reserveDoc();
+        reserveOneDoc();
         docState.doc = doc;
         docState.docID = numDocsInRAM;
         docCount++;

Modified: lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1662654&r1=1662653&r2=1662654&view=diff
==============================================================================
--- lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java Fri Feb 27 10:20:24 2015
@@ -194,7 +194,7 @@ import org.apache.lucene.util.Version;
 public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
 
   /** Hard limit on maximum number of documents that may be added to the
-   *  index.  If you try to add more than this you'll hit {@code IllegalStateException}. */
+   *  index.  If you try to add more than this you'll hit {@code IllegalArgumentException}. */
   // We defensively subtract 128 to be well below the lowest
   // ArrayUtil.MAX_ARRAY_LENGTH on "typical" JVMs.  We don't just use
   // ArrayUtil.MAX_ARRAY_LENGTH here because this can vary across JVMs:
@@ -813,6 +813,7 @@ public class IndexWriter implements Clos
       }
 
       rollbackSegments = segmentInfos.createBackupSegmentInfos();
+      pendingNumDocs.set(segmentInfos.totalDocCount());
 
       // start with previous field numbers, but new FieldInfos
       globalFieldNumberMap = getFieldNumberMap();
@@ -2241,13 +2242,16 @@ public class IndexWriter implements Clos
      */
     try {
       synchronized (fullFlushLock) { 
-        docWriter.lockAndAbortAll(this);
+        long abortedDocCount = docWriter.lockAndAbortAll(this);
+        pendingNumDocs.addAndGet(-abortedDocCount);
+        
         processEvents(false, true);
         synchronized (this) {
           try {
             // Abort any running merges
             abortMerges();
             // Remove all segments
+            pendingNumDocs.addAndGet(-segmentInfos.totalDocCount());
             segmentInfos.clear();
             // Ask deleter to locate unreferenced files & remove them:
             deleter.checkpoint(segmentInfos, false);
@@ -2263,6 +2267,7 @@ public class IndexWriter implements Clos
             ++changeCount;
             segmentInfos.changed();
             globalFieldNumberMap.clear();
+
             success = true;
           } finally {
             docWriter.unlockAllAfterAbortAll(this);
@@ -2510,6 +2515,8 @@ public class IndexWriter implements Clos
    * @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();
@@ -2529,23 +2536,30 @@ public class IndexWriter implements Clos
 
       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 = new SegmentInfos(); // read infos from dir
+        sis.read(dir);
+        totalDocCount += sis.totalDocCount();
+        commits.add(sis);
+      }
+
+      // Best-effort up front check:
+      testReserveDocs(totalDocCount);
 
       boolean success = false;
       try {
-        for (Directory dir : dirs) {
-          if (infoStream.isEnabled("IW")) {
-            infoStream.message("IW", "addIndexes: process directory " + dir);
-          }
-          SegmentInfos sis = new SegmentInfos(); // read infos from dir
-          sis.read(dir);
+        for (SegmentInfos sis : commits) {
 
           final Set<String> dsFilesCopied = new HashSet<>();
           final Map<String, String> dsNames = new HashMap<>();
           final Set<String> copiedFiles = new HashSet<>();
 
-          totalDocCount += sis.totalDocCount();
-
           for (SegmentCommitInfo info : sis) {
             assert !infos.contains(info): "dup info dir=" + info.info.dir + " name=" + info.info.name;
 
@@ -2567,12 +2581,7 @@ public class IndexWriter implements Clos
       } 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]));
           }
         }
       }
@@ -2581,9 +2590,10 @@ public class IndexWriter implements Clos
         success = false;
         try {
           ensureOpen();
-          // Make sure adding the new documents to this index won't
-          // exceed the limit:
+
+          // Now reserve the docs, just before we update SIS:
           reserveDocs(totalDocCount);
+
           success = true;
         } finally {
           if (!success) {
@@ -2649,12 +2659,16 @@ public class IndexWriter implements Clos
    *           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(IndexReader... readers) throws IOException {
     ensureOpen();
-    int numDocs = 0;
 
     try {
+      // long so we can detect int overflow:
+      long numDocs = 0;
+
       if (infoStream.isEnabled("IW")) {
         infoStream.message("IW", "flush at addIndexes(IndexReader...)");
       }
@@ -2669,11 +2683,10 @@ public class IndexWriter implements Clos
         }
       }
 
-      // Make sure adding the new documents to this index won't
-      // exceed the limit:
-      reserveDocs(numDocs);
-      
-      final IOContext context = new IOContext(new MergeInfo(numDocs, -1, true, -1));
+      // Best-effort up front check:
+      testReserveDocs(numDocs);
+
+      final IOContext context = new IOContext(new MergeInfo((int) numDocs, -1, true, -1));
 
       // TODO: somehow we should fix this merge so it's
       // abortable so that IW.close(false) is able to stop it
@@ -2761,6 +2774,10 @@ public class IndexWriter implements Clos
           return;
         }
         ensureOpen();
+
+        // Now reserve the docs, just before we update SIS:
+        reserveDocs(numDocs);
+      
         segmentInfos.add(infoPerCommit);
         checkpoint();
       }
@@ -4899,12 +4916,28 @@ public class IndexWriter implements Clos
 
   /** Anything that will add N docs to the index should reserve first to
    *  make sure it's allowed.  This will throw {@code
-   *  IllegalStateException} if it's not allowed. */ 
-  private void reserveDocs(int numDocs) {
-    if (pendingNumDocs.addAndGet(numDocs) > actualMaxDocs) {
-      // Reserve failed
-      pendingNumDocs.addAndGet(-numDocs);
-      throw new IllegalStateException("number of documents in the index cannot exceed " + actualMaxDocs);
+   *  IllegalArgumentException} if it's not allowed. */ 
+  private void reserveDocs(long addedNumDocs) {
+    assert addedNumDocs >= 0;
+    if (pendingNumDocs.addAndGet(addedNumDocs) > actualMaxDocs) {
+      // Reserve failed: put the docs back and throw exc:
+      pendingNumDocs.addAndGet(-addedNumDocs);
+      tooManyDocs(addedNumDocs);
     }
   }
+
+  /** Does a best-effort check, that the current index would accept this many additional docs, but does not actually reserve them.
+   *
+   * @throws IllegalArgumentException if there would be too many docs */
+  private void testReserveDocs(long addedNumDocs) {
+    assert addedNumDocs >= 0;
+    if (pendingNumDocs.get() + addedNumDocs > actualMaxDocs) {
+      tooManyDocs(addedNumDocs);
+    }
+  }
+
+  private void tooManyDocs(long addedNumDocs) {
+    assert addedNumDocs >= 0;
+    throw new IllegalArgumentException("number of documents in the index cannot exceed " + actualMaxDocs + " (current document count is " + pendingNumDocs.get() + "; added numDocs is " + addedNumDocs + ")");
+  }
 }

Modified: lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java?rev=1662654&r1=1662653&r2=1662654&view=diff
==============================================================================
--- lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java Fri Feb 27 10:20:24 2015
@@ -342,6 +342,7 @@ public final class SegmentInfos implemen
     try {
       final int format = input.readInt();
       final int actualFormat;
+      long totalDocs = 0;
       if (format == CodecUtil.CODEC_MAGIC) {
         // 4.0+
         actualFormat = CodecUtil.checkHeaderNoMagic(input, "segments", VERSION_40, VERSION_49);
@@ -357,6 +358,7 @@ public final class SegmentInfos implemen
           //System.out.println("SIS.read seg=" + seg + " codec=" + codec);
           SegmentInfo info = codec.segmentInfoFormat().getSegmentInfoReader().read(directory, segName, IOContext.READ);
           info.setCodec(codec);
+          totalDocs += info.getDocCount();
           long delGen = input.readLong();
           int delCount = input.readInt();
           if (delCount < 0 || delCount > info.getDocCount()) {
@@ -415,6 +417,7 @@ public final class SegmentInfos implemen
         Codec codec = Codec.forName("Lucene3x");
         for (SegmentCommitInfo info : this) {
           info.info.setCodec(codec);
+          totalDocs += info.info.getDocCount();
         }
       }
 
@@ -429,6 +432,11 @@ public final class SegmentInfos implemen
         CodecUtil.checkEOF(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 + " (resource: " + input + ")");
+      }
+
       success = true;
     } finally {
       if (!success) {
@@ -1164,11 +1172,13 @@ public final class SegmentInfos implemen
   /** 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 (int) count;
   }
 
   /** Call this before committing if changes have been made to the

Modified: lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java?rev=1662654&r1=1662653&r2=1662654&view=diff
==============================================================================
--- lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java (original)
+++ lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java Fri Feb 27 10:20:24 2015
@@ -56,18 +56,23 @@ final class StandardDirectoryReader exte
         SegmentInfos sis = new SegmentInfos();
         sis.read(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), termInfosIndexDivisor, IOContext.READ);
-            success = true;
-          } finally {
-            if (!success) {
-              IOUtils.closeWhileHandlingException(readers);
-            }
+          }
+
+          // This may throw IllegalArgumentException 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, termInfosIndexDivisor, false);
+          success = true;
+
+          return reader;
+        } finally {
+          if (success == false) {
+            IOUtils.closeWhileHandlingException(readers);
           }
         }
-        return new StandardDirectoryReader(directory, readers, null, sis, termInfosIndexDivisor, false);
       }
     }.run(commit);
   }

Modified: lucene/dev/branches/lucene_solr_4_10/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_10/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java?rev=1662654&r1=1662653&r2=1662654&view=diff
==============================================================================
--- lucene/dev/branches/lucene_solr_4_10/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java (original)
+++ lucene/dev/branches/lucene_solr_4_10/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java Fri Feb 27 10:20:24 2015
@@ -17,8 +17,10 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
+import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.concurrent.CountDownLatch;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -31,6 +33,7 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.util.LuceneTestCase.Monster;
 import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.TimeUnits;
 import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
 
@@ -90,7 +93,7 @@ public class TestIndexWriterMaxDocs exte
       try {
         w.addDocument(new Document());
         fail("didn't hit exception");
-      } catch (IllegalStateException ise) {
+      } catch (IllegalArgumentException iae) {
         // expected
       }
       w.close();
@@ -113,7 +116,7 @@ public class TestIndexWriterMaxDocs exte
       try {
         w.addDocuments(Collections.singletonList(new Document()));
         fail("didn't hit exception");
-      } catch (IllegalStateException ise) {
+      } catch (IllegalArgumentException iae) {
         // expected
       }
       w.close();
@@ -136,7 +139,7 @@ public class TestIndexWriterMaxDocs exte
       try {
         w.updateDocument(new Term("field", "foo"), new Document());
         fail("didn't hit exception");
-      } catch (IllegalStateException ise) {
+      } catch (IllegalArgumentException iae) {
         // expected
       }
       w.close();
@@ -159,7 +162,7 @@ public class TestIndexWriterMaxDocs exte
       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();
@@ -198,7 +201,7 @@ public class TestIndexWriterMaxDocs exte
       try {
         w.addDocument(new Document());
         fail("didn't hit exception");
-      } catch (IllegalStateException ise) {
+      } catch (IllegalArgumentException iae) {
         // expected
       }
       w.close();
@@ -244,7 +247,7 @@ public class TestIndexWriterMaxDocs exte
       try {
         w.addDocument(new Document());
         fail("didn't hit exception");
-      } catch (IllegalStateException ise) {
+      } catch (IllegalArgumentException iae) {
         // expected
       }
       w.close();
@@ -270,7 +273,7 @@ public class TestIndexWriterMaxDocs exte
       try {
         w2.addIndexes(new Directory[] {dir});
         fail("didn't hit exception");
-      } catch (IllegalStateException ise) {
+      } catch (IllegalArgumentException iae) {
         // expected
       }
       assertEquals(1, w2.maxDoc());
@@ -278,7 +281,7 @@ public class TestIndexWriterMaxDocs exte
       try {
         w2.addIndexes(new IndexReader[] {ir});
         fail("didn't hit exception");
-      } catch (IllegalStateException ise) {
+      } catch (IllegalArgumentException iae) {
         // expected
       }
       w2.close();
@@ -366,7 +369,7 @@ public class TestIndexWriterMaxDocs exte
     dir.close();
     dir2.close();
   }
-
+  
   public void testTooLargeMaxDocs() throws Exception {
     try {
       IndexWriter.setMaxDocs(Integer.MAX_VALUE);
@@ -375,4 +378,244 @@ public class TestIndexWriterMaxDocs exte
       // expected
     }
   }
+
+  // LUCENE-6299
+  public void testDeleteAll() throws Exception {
+    setIndexWriterMaxDocs(1);
+    try {
+      Directory dir = newDirectory();
+      IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(TEST_VERSION_CURRENT, 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(TEST_VERSION_CURRENT, 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(TEST_VERSION_CURRENT, 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();
+      final IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(TEST_VERSION_CURRENT, null));
+
+      final 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(TEST_VERSION_CURRENT, null));
+      w.addDocument(new Document());
+      w.close();
+      w = new IndexWriter(dir, new IndexWriterConfig(TEST_VERSION_CURRENT, 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(TEST_VERSION_CURRENT, null));
+      w.addDocument(new Document());
+      w.close();
+      w = new IndexWriter(dir, new IndexWriterConfig(TEST_VERSION_CURRENT, 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(TEST_VERSION_CURRENT, 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(TEST_VERSION_CURRENT, null));
+    w.addDocument(new Document());
+    w.addDocument(new Document());
+    w.close();
+
+    setIndexWriterMaxDocs(1);
+    try {       
+      new IndexWriter(dir, new IndexWriterConfig(TEST_VERSION_CURRENT, null));
+      fail("didn't hit exception");
+    } catch (CorruptIndexException cie) {
+      // expected
+    } finally {
+      restoreIndexWriterMaxDocs();
+    }
+
+    dir.close();
+  }
 }